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 2016/03/16 15:56:09 UTC

[1/5] gora git commit: GORA-362 Refactor gora-dynamodb to support avro serialization

Repository: gora
Updated Branches:
  refs/heads/master d46a96f41 -> 96e623b03


http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBStore.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBStore.java b/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBStore.java
deleted file mode 100644
index 4c08428..0000000
--- a/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBStore.java
+++ /dev/null
@@ -1,386 +0,0 @@
-/**
- * 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;
-
-import java.io.IOException;
-import java.util.HashSet;
-
-import static junit.framework.Assert.assertTrue;
-import static junit.framework.Assert.assertFalse;
-
-import org.apache.gora.dynamodb.query.DynamoDBKey;
-import org.apache.gora.dynamodb.query.DynamoDBQuery;
-import org.apache.gora.examples.generated.person;
-import org.apache.gora.query.Query;
-import org.apache.gora.query.Result;
-import org.apache.gora.store.DataStore;
-import org.apache.gora.store.WSDataStoreTestBase;
-import org.junit.*;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.amazonaws.services.dynamodb.model.ComparisonOperator;
-
-/**
- * Test case for DynamoDBStore.
- */
-public class TestDynamoDBStore extends WSDataStoreTestBase<DynamoDBKey, person> {
-
-  public static final Logger log = LoggerFactory.getLogger(TestDynamoDBStore.class);
-
-  static {
-    setTestDriver(new GoraDynamoDBTestDriver());
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    setPersistentKeyClass(DynamoDBKey.class);
-    setPersistentValClass(person.class);
-    super.setUp();
-  }
-
-  public GoraDynamoDBTestDriver getTestDriver() {
-    return (GoraDynamoDBTestDriver) testDriver;
-  }
-
-//============================================================================
-    //We need to skip the following tests for a while until we fix some issues..
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testQueryStartKey() throws IOException {
-    log.info("test method: TestQueryStartKey SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testQueryEndKey() throws IOException {
-    log.info("test method: TestQueryEndKey SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testDeleteByQueryFields() throws IOException {
-    log.info("test method: TestDeleteByQueryFields SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testNewInstance() throws IOException, Exception {
-    log.info("test method: TestNewInstance SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testAutoCreateSchema() throws Exception {
-    log.info("test method: TestAutoCreateSchema SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testTruncateSchema() throws Exception {
-    log.info("test method: TestTruncateSchema SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testPutNested() throws IOException, Exception {
-    log.info("test method: TestPutNested SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testPutArray() throws IOException, Exception {
-    log.info("test method: TestPutArray SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testPutBytes() throws IOException, Exception {
-    log.info("test method: TestPutBytes SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testPutMap() throws IOException, Exception {
-    log.info("test method: TestPutMap SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testEmptyUpdate() throws IOException, Exception {
-    log.info("test method: TestEmptyUpdate SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testDeleteSchema() throws IOException, Exception {
-    log.info("test method: TestDeleteSchema SKIPPED.");
-  }
-  @Ignore("Needs to be skipped for a while until some issues are fixed")
-  @Override
-  public void testGetWithFields() throws IOException, Exception {
-    log.info("test method: TestGetWithFields SKIPPED.");
-  }
-//==========================================================================
-
-  /**
-   * Tests deleting items using a query
-   */
-  @Override
-  public void assertTestDeleteByQueryDataStore(){
-    try {
-      log.info("test method: TestDeleteByQuery using DynamoDB store.");
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("NOWHERE");
-      dKey.setRangeKey("10/10/1880");
-      person p1 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "John", "Doe", "Peru", "Brazil", "Ecuador");
-      dataStore.put(dKey, p1);
-      dKey.setRangeKey("11/10/1707");
-      person p2 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "Juan", "Perez", "Germany", "USA", "Scotland");
-      dataStore.put(dKey, p2);
-      DynamoDBQuery.setScanCompOp(ComparisonOperator.LE);
-      DynamoDBQuery.setType(DynamoDBQuery.SCAN_QUERY);
-      Query<DynamoDBKey, person> query = new DynamoDBQuery<DynamoDBKey, person>();
-      query.setKey(dKey);
-      log.info("Number of records deleted: "+ dataStore.deleteByQuery(query));
-    } catch (Exception e) {
-      log.error("Error while running test: TestDeleteByQuery");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Tests updating a specific item
-   */
-  @Override
-  public void assertTestUpdateDataStore(){
-    try{
-      log.info("test method: TestUpdate using DynamoDB store.");
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("Peru");
-      dKey.setRangeKey("10/10/1880");
-      person p1 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
-      dataStore.put(dKey, p1);
-      p1.setFirstName("Ataucuri");
-      dataStore.put(dKey, p1);
-    } catch (Exception e){
-      log.error("error in test method: testUpdate.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to test deleting a schema
-   */
-  @Override
-  public void assertDeleteSchema(){
-    try {
-      log.info("test method: TestDeleteSchema using DynamoDB store.");
-      dataStore.deleteSchema();
-    } catch (Exception e) {
-      log.error("error in test method: testDeleteSchema.");
-      e.printStackTrace();
-    }
-  }
- 
-  /**
-   * Method to verify if a schema exists or not
-   */
-  @Override
-  public void assertSchemaExists(String schemaName) throws Exception {
-    log.info("test method: TestSchemaExists using DynamoDB store.");
-    assertTrue(dataStore.schemaExists());
-  }
-
-  /**
-   * Method to put items into the data store
-   */
-  @Override
-  public void assertPut(){
-    try {
-      log.info("test method: TestPut using DynamoDB store.");
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("Peru");
-      dKey.setRangeKey("10/10/1880");
-      person p1 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
-      dataStore.put(dKey, p1);
-      dKey.setRangeKey("11/10/1707");
-      person p2 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "William", "Wallace", "Germany", "USA", "Scotland");
-      dataStore.put(dKey, p2);
-    } catch (Exception e) {
-      log.error("error in test method: testPut.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to query the data store
-   */
-  @Override
-  public void assertTestQueryDataStore(){
-    log.info("test method: testQuery using DynamoDB store.");
-    try {
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("Peru");
-      DynamoDBQuery.setScanCompOp(ComparisonOperator.LE);
-      DynamoDBQuery.setType(DynamoDBQuery.SCAN_QUERY);
-      Query<DynamoDBKey, person> query = new DynamoDBQuery<DynamoDBKey, person>();
-      query.setKey(dKey);
-      Result<DynamoDBKey, person> queryResult = dataStore.execute(query);
-      processQueryResult(queryResult);
-    } catch (Exception e) {
-      log.error("error in test method: testQuery.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to query items into the data store
-   */
-  @Override
-  public void assertTestQueryKeyRange(){
-    log.info("test method: testQueryKeyRange using specific data store.");
-    try {
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      DynamoDBKey<String, String> startKey = new DynamoDBKey<String, String>();
-      DynamoDBKey<String, String> endKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("Peru");
-      startKey.setRangeKey("01/01/1700");
-      endKey.setRangeKey("31/12/1900");
-      DynamoDBQuery.setRangeCompOp(ComparisonOperator.BETWEEN);
-      DynamoDBQuery.setType(DynamoDBQuery.RANGE_QUERY);
-      Query<DynamoDBKey, person> query = new DynamoDBQuery<DynamoDBKey, person>();
-      query.setKey(dKey);
-      query.setStartKey(startKey);
-      query.setEndKey(endKey);
-      Result<DynamoDBKey, person> queryResult = dataStore.execute(query);
-      processQueryResult(queryResult);
-    } catch (Exception e) {
-      log.error("error in test method: testQueryKeyRange.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to get an specific object using a key
-   */
-  @Override
-  public void assertTestGetDataStore(){
-    log.info("test method: testGet using specific data store.");
-    try {
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("USA");
-      dKey.setRangeKey("10/10/1999");
-      // insert item
-      person p1 = buildPerson(dKey.getHashKey().toString(), dKey.getRangeKey().toString(), "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
-      dataStore.put(dKey, p1);
-      // get item
-      person p2 = dataStore.get(dKey);
-      printPersonInfo(p2);
-    } catch (Exception e) {
-      log.error("error in test method: testGetDataStore.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to delete items into the data store
-   */
-  @Override
-  public void assertTestDeleteDataStore() {
-    log.info("test method: testDelete by key");
-    try {
-      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
-      dKey.setHashKey("Brazil");
-      dKey.setRangeKey("10/10/1985");
-      person p1 = new person();
-      p1.setHashKey(dKey.getHashKey());
-      p1.setRangeKey(dKey.getRangeKey());
-      p1.setFirstName("Joao");
-      p1.setLastName("Velasco");
-      dataStore.put(dKey, p1);
-      assertTrue(dataStore.delete(dKey));
-      dKey.setRangeKey("10/10/1000");
-      assertFalse(dataStore.delete(dKey));
-    }catch (Exception e) {
-      log.error("error in test method: testDeleteDataStore.");
-      e.printStackTrace();
-    }
-  }
-
-  /**
-   * Method to create the data store
-   */
-  @Override
-  protected DataStore<DynamoDBKey, person> createDataStore() {
-    log.info("Creating DynamoDB data store.");
-    try {
-      dataStore = getTestDriver().getDataStore();
-      dataStore.createSchema();
-    } catch (Exception e) {
-      log.error("error while creating DynamoDB data store");
-      e.printStackTrace();
-    }
-      return dataStore;
-  }
-
-  /**
-   * Processes query results from an query execution
-   * @param pQueryResult
-   */
-  private void processQueryResult(Result<DynamoDBKey, person> pQueryResult){
-    try {
-      log.debug("Processing tests results.");
-      while(pQueryResult.next())
-        printPersonInfo(pQueryResult.get());
-      } catch (IOException e) {
-        log.error("error while processing tests results.");
-        e.printStackTrace();
-      } catch (Exception e) {
-        log.error("error while processing tests results.");
-        e.printStackTrace();
-      }
-    }
-
-  /**
-   * Method to generate persisten objects
-   * @param key
-   * @param pRangeKey
-   * @param pFirstName
-   * @param pLastName
-   * @param places
-   * @return
-   */
-  private person buildPerson(String key, String pRangeKey, String pFirstName, String pLastName, String ...places){
-    person newPerson = new person();
-    newPerson.setRangeKey(pRangeKey);
-    newPerson.setHashKey(key);
-    newPerson.setFirstName(pFirstName);
-    newPerson.setLastName(pLastName);
-    newPerson.setVisitedplaces(new HashSet<String>());
-    for(String place : places)
-      newPerson.getVisitedplaces().add(place);
-      return newPerson;
-    }
-
-  /**
-   * Method to print the object returned from Get method
-   * @param pPerson
-   */
-  private void printPersonInfo(person pPerson){
-    log.info("Origin:\t" + pPerson.getHashKey() +
-      "\n Birthday:\t" + pPerson.getRangeKey() +
-      "\n FirstName:" + pPerson.getFirstName() +
-      "\n LastName:" + pPerson.getLastName() +
-      "\n Visited Places:");
-    for(String place : pPerson.getVisitedplaces())
-      log.info("\t" + place);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-mongodb/src/main/java/org/apache/gora/mongodb/store/MongoStore.java
----------------------------------------------------------------------
diff --git a/gora-mongodb/src/main/java/org/apache/gora/mongodb/store/MongoStore.java b/gora-mongodb/src/main/java/org/apache/gora/mongodb/store/MongoStore.java
index 5e0b866..79c5e91 100644
--- a/gora-mongodb/src/main/java/org/apache/gora/mongodb/store/MongoStore.java
+++ b/gora-mongodb/src/main/java/org/apache/gora/mongodb/store/MongoStore.java
@@ -38,7 +38,6 @@ import org.apache.gora.mongodb.query.MongoDBQuery;
 import org.apache.gora.mongodb.query.MongoDBResult;
 import org.apache.gora.mongodb.utils.BSONDecorator;
 import org.apache.gora.mongodb.utils.GoraDBEncoder;
-import org.apache.gora.persistency.Persistent;
 import org.apache.gora.persistency.impl.BeanFactoryImpl;
 import org.apache.gora.persistency.impl.DirtyListWrapper;
 import org.apache.gora.persistency.impl.DirtyMapWrapper;
@@ -573,7 +572,7 @@ public class MongoStore<K, T extends PersistentBase> extends
       clazz = ClassLoadingUtils.loadClass(fieldSchema.getFullName());
     } catch (ClassNotFoundException e) {
     }
-    Persistent record = new BeanFactoryImpl(keyClass, clazz).newPersistent();
+    PersistentBase record = (PersistentBase) new BeanFactoryImpl(keyClass, clazz).newPersistent();
     for (Field recField : fieldSchema.getFields()) {
       Schema innerSchema = recField.schema();
       DocumentFieldType innerStoreType = mapping

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index cbf3be2..946a569 100644
--- a/pom.xml
+++ b/pom.xml
@@ -702,7 +702,7 @@
     <module>gora-goraci</module>
     <module>gora-hbase</module>
     <!-- module>gora-lucene</module -->
-    <!--module>gora-dynamodb</module -->
+    <module>gora-dynamodb</module>
     <!--module>gora-sql</module -->
     <module>gora-maven-plugin</module>
     <module>gora-mongodb</module>
@@ -730,7 +730,7 @@
     <hbase.test.version>0.98.8-hadoop2</hbase.test.version>
     <cxf-rt-frontend-jaxrs.version>2.5.2</cxf-rt-frontend-jaxrs.version>
     <!-- Amazon Dependencies -->
-    <amazon.version>1.3.12</amazon.version>
+    <amazon.version>1.10.55</amazon.version>
     <!-- Cassandra Dependencies -->
     <cassandra.version>2.0.2</cassandra.version>
     <hector.version>1.1-4</hector.version>


[3/5] gora git commit: GORA-362 Refactor gora-dynamodb to support avro serialization

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java
index e445d51..699d051 100644
--- a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBQuery.java
@@ -20,6 +20,12 @@ package org.apache.gora.dynamodb.query;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
 
 import org.apache.gora.filter.Filter;
 import org.apache.gora.persistency.Persistent;
@@ -29,13 +35,12 @@ import org.apache.gora.store.DataStore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBQueryExpression;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBScanExpression;
-import com.amazonaws.services.dynamodb.model.AttributeValue;
-import com.amazonaws.services.dynamodb.model.ComparisonOperator;
-import com.amazonaws.services.dynamodb.model.Condition;
-import com.amazonaws.services.dynamodb.model.KeySchema;
-import com.amazonaws.services.dynamodb.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBScanExpression;
+import com.amazonaws.services.dynamodbv2.model.AttributeValue;
+import com.amazonaws.services.dynamodbv2.model.ComparisonOperator;
+import com.amazonaws.services.dynamodbv2.model.Condition;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeyType;
 
 public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
@@ -69,6 +74,8 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
    */
   public static final String SCAN_QUERY = "scan";
 
+  public static final ComparisonOperator DEFAULT_SCAN_OP = ComparisonOperator.GE;
+
   /**
    * Query type property
    */
@@ -88,26 +95,29 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
   /**
    * Key schema used for the query
    */
-  private KeySchema keySchema;
+  private ArrayList<KeySchemaElement> keySchema;
 
   /**
    * Hash key used for the query
    */
   private K hashKey;
 
+  private Map<String, String> keyItems;
+
   /**
    * Default Constructor
    */
   public DynamoDBQuery(){
-  super(null);
+    super(null);
   }
-  
+
   /**
    * Constructor
+   * 
    * @param dataStore
    */
   public DynamoDBQuery(DataStore<K, T> dataStore) {
-  super(dataStore);
+    super(dataStore);
   }
 
   /**
@@ -126,153 +136,213 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
     return this.hashKey;
   }
 
+  private void defineQueryParams() {
+    if ((query.getStartKey() != null || query.getKey() != null)
+        && query.getEndKey() != null) {
+      DynamoDBQuery.setType(RANGE_QUERY);
+    } else if (query.getKey() != null || query.getStartKey() != null) {
+      DynamoDBQuery.setType(SCAN_QUERY);
+    }
+  }
+
   /**
    * Builds query expression depending on query type (range or scan) 
    */
-  public void buildExpression(){
-    AttributeValue hashAttrValue = buildKeyHashAttribute();
-    if (hashAttrValue == null)
-      throw new IllegalStateException("There is not a key schema defined.");
-    if (DynamoDBQuery.getType().equals(RANGE_QUERY)){
-      Condition newCondition = buildRangeCondition();
-      buildQueryExpression(newCondition, hashAttrValue);
+  public void buildExpression() {
+    defineQueryParams();
+    if (DynamoDBQuery.getType().equals(RANGE_QUERY)) {
+      buildRangeExpression();
+    } else if (DynamoDBQuery.getType().equals(SCAN_QUERY)) {
+      buildScanExpression();
+    } else {
+      throw new IllegalArgumentException("Query type not supported");
     }
-    if (DynamoDBQuery.getType().equals(SCAN_QUERY))
-      buildScanExpression(hashAttrValue);
   }
 
   /**
-   * Builds scan query expression using a hash attribute value where to start
-   * @param pHashAttrValueHash attribute value where to start scanning
+   * Builds hash key attribute from generic query received.
+   * 
+   * @param qKey
+   * 
+   * @returnAttributeValue build from query
    */
-  public void buildScanExpression(AttributeValue pHashAttrValue){
-    DynamoDBScanExpression newScanExpression = new DynamoDBScanExpression();
-    // TODO right now we only support scanning using the key, but we should support other types of scans
-    newScanExpression.addFilterCondition(getKeySchema().getHashKeyElement().getAttributeName(), buildKeyScanCondition());
-    dynamoDBExpression = newScanExpression;
+  private Map<String, AttributeValue> buildHashKey(K qKey) {
+    Map<String, AttributeValue> hashKey = new HashMap<>();
+    for (KeySchemaElement key : getKeySchema()) {
+      AttributeValue attr = new AttributeValue();
+      if (key.getKeyType().equals(KeyType.HASH.toString())) {
+        if (keyItems.get(key.getAttributeName()).equals("N")) {
+          attr.withN(getHashKey(qKey).toString());
+        } else if (keyItems.get(key.getAttributeName()).equals("S")) {
+          attr.withS(getHashKey(qKey).toString());
+        } else if (keyItems.get(key.getAttributeName()).equals("B")) {
+          attr.withB(ByteBuffer.wrap(getHashKey(qKey).toString().getBytes(Charset.defaultCharset())));
+        } else {
+          throw new IllegalArgumentException("Data type not supported for "
+              + key.getAttributeName());
+        }
+        hashKey.put(key.getAttributeName(), attr);
+      }
+    }
+    if (hashKey.isEmpty()) {
+      throw new IllegalStateException("No key value has been defined.");
+    }
+    return hashKey;
   }
 
   /**
-   * Builds range query expression
-   * @param pNewConditionCondition for querying
-   * @param pHashAttrValueHash attribute value where to start
-   */
-  public void buildQueryExpression(Condition pNewCondition, AttributeValue pHashAttrValue) {
-    DynamoDBQueryExpression newQueryExpression = new DynamoDBQueryExpression(pHashAttrValue); 
-    newQueryExpression.setConsistentRead(getConsistencyReadLevel());
-    newQueryExpression.setRangeKeyCondition(pNewCondition);
-    dynamoDBExpression = newQueryExpression;
+   * Builds range key attribute from generic query received.
+   * 
+   * @param qKey
+   * 
+   * @return
+   */
+  private Map<String, AttributeValue> buildRangeKey(K qKey) {
+    Map<String, AttributeValue> kAttrs = new HashMap<>();
+    for (KeySchemaElement key : getKeySchema()) {
+      AttributeValue attr = new AttributeValue();
+      if (key.getKeyType().equals(KeyType.RANGE.toString())) {
+        if (keyItems.get(key.getAttributeName()).equals("N")) {
+          attr.withN(getRangeKey(qKey).toString());
+        } else if (keyItems.get(key.getAttributeName()).equals("S")) {
+          attr.withS(getRangeKey(qKey).toString());
+        } else if (keyItems.get(key.getAttributeName()).equals("B")) {
+          attr.withB(ByteBuffer.wrap(getRangeKey(qKey).toString().getBytes(Charset.defaultCharset())));
+        } else {
+          throw new IllegalArgumentException("Data type not supported for "
+              + key.getAttributeName());
+        }
+        kAttrs.put(key.getAttributeName(), attr);
+      }
+    }
+    return kAttrs;
   }
 
   /**
-   * Builds hash key attribute from generic query received
-   * @returnAttributeValue build from query
-   */
-  private AttributeValue buildKeyHashAttribute(){
-    String pAttrType = getKeySchema().getHashKeyElement().getAttributeType();
-    if(pAttrType.equals("S"))
-      return new AttributeValue().withS(getHashKey(query.getKey()).toString());
-    else if(pAttrType.equals("N"))
-      return new AttributeValue().withN(getHashKey(query.getKey()).toString());
-    return null;
+   * Builds scan query expression using a hash attribute value where to start
+   * 
+   * @param pHashAttrValueHash
+   *          attribute value where to start scanning
+   */
+  public void buildScanExpression() {
+    K qKey = getKey();
+    if (qKey == null) {
+      LOG.warn("No key defined. Trying with startKey.");
+      qKey = query.getStartKey();
+      if (qKey == null) {
+        throw new IllegalStateException("No key has been defined please check");
+      }
+    }
+    ComparisonOperator compOp = getScanCompOp() != null ? getScanCompOp()
+        : DEFAULT_SCAN_OP;
+
+    DynamoDBScanExpression newScanExpression = new DynamoDBScanExpression();
+    // hash key condition
+    Map<String, AttributeValue> hashAttrVals = buildHashKey(qKey);
+    for (Entry<String, AttributeValue> en : hashAttrVals.entrySet()) {
+      Condition scanFilterHashCondition = new Condition().withComparisonOperator(
+          compOp.toString()).withAttributeValueList(en.getValue());
+      newScanExpression.addFilterCondition(en.getKey(), scanFilterHashCondition);
+    }
+    // range key condition
+    Map<String, AttributeValue> rangeAttrVals = buildRangeKey(qKey);
+    for (Entry<String, AttributeValue> en : rangeAttrVals.entrySet()) {
+      Condition scanFilterRangeCondition = new Condition().withComparisonOperator(
+          compOp.toString()).withAttributeValueList(en.getValue());
+      newScanExpression.addFilterCondition(en.getKey(), scanFilterRangeCondition);
+    }
+    dynamoDBExpression = newScanExpression;
+  }
+
+  /**
+   * Builds range query expression
+   * 
+   */
+  public void buildRangeExpression() {
+    DynamoDBScanExpression queryExpression = new DynamoDBScanExpression();
+    ComparisonOperator compOp = ComparisonOperator.BETWEEN;
+    // hash key range
+    Map<String, AttributeValue> hashAttrVals = buildHashKey(query.getStartKey());
+    Map<String, AttributeValue> endHashAttrVals = buildHashKey(query.getEndKey());
+    for (Entry<String, AttributeValue> en : hashAttrVals.entrySet()) {
+      Condition scanFilterHashCondition = new Condition().withComparisonOperator(
+          compOp.toString()).withAttributeValueList(en.getValue(), endHashAttrVals.get(en.getKey()));
+      queryExpression.addFilterCondition(en.getKey(), scanFilterHashCondition);
+    }
+    // range key range
+    Map<String, AttributeValue> rangeAttrVals = buildRangeKey(query.getStartKey());
+    Map<String, AttributeValue> endRangeAttrVals = buildRangeKey(query.getEndKey());
+    for (Entry<String, AttributeValue> en : rangeAttrVals.entrySet()) {
+      Condition scanFilterRangeCondition = new Condition().withComparisonOperator(
+          compOp.toString()).withAttributeValueList(en.getValue(), endRangeAttrVals.get(en.getKey()));
+      queryExpression.addFilterCondition(en.getKey(), scanFilterRangeCondition);
+    }
+    dynamoDBExpression = queryExpression;
   }
 
   /**
    * Gets hash key for querying
+   * 
    * @param key
    * @return
    */
   private Object getHashKey(K key){
     Object hashKey = null;
     try {
-    // Our key may be have hash and range keys
-    for (Method met :key.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getHashKey")){
-        Object [] params = null;
-        hashKey = met.invoke(key, params);
-        break;
+      // Our key may be have hash and range keys
+      for (Method met : key.getClass().getDeclaredMethods()) {
+        if (met.getName().equals("getHashKey")) {
+          Object[] params = null;
+          hashKey = met.invoke(key, params);
+          break;
+        }
       }
+    } catch (IllegalArgumentException e) {
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new IllegalArgumentException(e);
+    } catch (IllegalAccessException e) {
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new RuntimeException(e);
+    } catch (InvocationTargetException e) {
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new RuntimeException(e);
     }
-  } catch (IllegalArgumentException e) {
-    LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
-  } catch (IllegalAccessException e) {
-    LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
-  } catch (InvocationTargetException e) {
-    LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
-  }
-  return hashKey;
+    return hashKey;
   }
 
   /**
    * Gets range key for querying from generic query object received
+   * 
    * @param key
    * @return
    */
   private Object getRangeKey(K key){
     Object rangeKey = null;
     try {
-        // Our key may be have hash and range keys
+      // Our key may be have hash and range keys
       for (Method met :key.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getRangeKey")){
-        Object [] params = null;
-        rangeKey = met.invoke(key, params);
-        break;
+        if(met.getName().equals("getRangeKey")){
+          Object [] params = null;
+          rangeKey = met.invoke(key, params);
+          break;
         }
       }
     } catch (IllegalArgumentException e) {
-      LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new IllegalArgumentException(e);
     } catch (IllegalAccessException e) {
-    LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new RuntimeException(e);
     } catch (InvocationTargetException e) {
-    LOG.info("DynamoDBStore: Error while trying to fetch range key.");
-    e.printStackTrace();
+      LOG.info("DynamoDBStore: Error while trying to fetch range key.", e.getMessage());
+      throw new RuntimeException(e);
     }
     return rangeKey;
   }
 
   /**
-   * Builds key scan condition using scan comparator, and hash key attribute
-   * @return
-   */
-  private Condition buildKeyScanCondition(){
-  Condition scanKeyCondition = new Condition();
-  scanKeyCondition.setComparisonOperator(getScanCompOp());
-  scanKeyCondition.withAttributeValueList(buildKeyHashAttribute());
-  return scanKeyCondition;
-  }
-
-  /**
-   * Builds range condition based on elements set 
-   * @return
-   */
-  private Condition buildRangeCondition(){
-  KeySchemaElement kRangeSchema = getKeySchema().getRangeKeyElement();
-  Condition rangeKeyCondition = null;
-  if(kRangeSchema != null){
-    rangeKeyCondition = new Condition();
-    rangeKeyCondition.setComparisonOperator(ComparisonOperator.BETWEEN.toString());
-    AttributeValue startVal = null, endVal = null;
-    //startVal = buildKeyHashAttribute();
-    if(kRangeSchema.getAttributeType().equals("S")){
-      startVal = new AttributeValue().withS(getRangeKey(query.getStartKey()).toString());
-      endVal = new AttributeValue().withS(getRangeKey(query.getEndKey()).toString());
-    }
-    else if (kRangeSchema.getAttributeType().equals("N")){
-      startVal = new AttributeValue().withN(getRangeKey(query.getStartKey()).toString());
-      endVal = new AttributeValue().withN(getRangeKey(query.getEndKey()).toString());
-    }
-    rangeKeyCondition.withAttributeValueList(startVal, endVal);
-  }
-  return rangeKeyCondition;
-  }
-
-  /**
    * Gets read consistency level
+   * 
    * @return
    */
   public boolean getConsistencyReadLevel(){
@@ -281,6 +351,7 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Sets read consistency level
+   * 
    * @param pConsistencyReadLevel
    */
   public void setConsistencyReadLevel(boolean pConsistencyReadLevel){
@@ -289,14 +360,16 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Gets key schema
+   * 
    * @return
    */
-  public KeySchema getKeySchema(){
+  public ArrayList<KeySchemaElement> getKeySchema(){
     return keySchema;
   }
 
   /**
    * Gets query expression for query
+   * 
    * @return
    */
   public Object getQueryExpression(){
@@ -305,22 +378,26 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Sets query key schema used for queying
-   * @param pKeySchema
+   * 
+   * @param arrayList
    */
-  public void setKeySchema(KeySchema pKeySchema){
-    this.keySchema = pKeySchema;
+  public void setKeySchema(ArrayList<KeySchemaElement> arrayList) {
+    this.keySchema = arrayList;
   }
 
   /**
    * Sets query to be performed
+   * 
    * @param pQuery
    */
   public void setQuery(Query<K, T> pQuery){
-    this.query = pQuery;
+    this.setStartKey(query.getStartKey());
+    this.setEndKey(query.getEndKey());
   }
 
   /**
    * Gets query performed
+   * 
    * @return
    */
   public Query<K, T> getQuery(){
@@ -329,6 +406,7 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Gets query type
+   * 
    * @return
    */
   public static String getType() {
@@ -337,6 +415,7 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Sets query type
+   * 
    * @param pType
    */
   public static void setType(String pType) {
@@ -345,16 +424,16 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Gets scan comparator operator
+   * 
    * @return
    */
   public static ComparisonOperator getScanCompOp() {
-    if (scanCompOp == null)
-      scanCompOp = ComparisonOperator.GE;
     return scanCompOp;
   }
 
   /**
    * Sets scan query comparator operator
+   * 
    * @param scanCompOp
    */
   public static void setScanCompOp(ComparisonOperator scanCompOp) {
@@ -363,6 +442,7 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
 
   /**
    * Gets range query comparator operator
+   * 
    * @return
    */
   public static ComparisonOperator getRangeCompOp(){
@@ -379,10 +459,19 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
     rangeCompOp = pRangeCompOp;
   }
 
+  /**
+   * Sets the keyItems that could be used.
+   * 
+   * @param items
+   */
+  public void setKeyItems(Map<String, String> items) {
+    keyItems = items;
+  }
+
   @Override
   public void setFilter(Filter<K, T> filter) {
     // TODO Auto-generated method stub
-    
+
   }
 
   @Override
@@ -394,7 +483,7 @@ public class DynamoDBQuery<K, T extends Persistent> extends QueryWSBase<K, T> {
   @Override
   public void setLocalFilterEnabled(boolean enable) {
     // TODO Auto-generated method stub
-    
+
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBAvroStore.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBAvroStore.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBAvroStore.java
new file mode 100644
index 0000000..98f64ae
--- /dev/null
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBAvroStore.java
@@ -0,0 +1,136 @@
+/*
+ * 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.IOException;
+import java.util.Collections;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.gora.persistency.Persistent;
+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;
+import org.apache.gora.store.impl.DataStoreBase;
+
+public class DynamoDBAvroStore<K, T extends PersistentBase> extends
+DataStoreBase<K, T> implements IDynamoDB<K, T> {
+
+  /**
+   * The values are Avro fields pending to be stored.
+   *
+   * We want to iterate over the keys in insertion order. We don't want to lock
+   * the entire collection before iterating over the keys, since in the meantime
+   * other threads are adding entries to the map.
+   */
+  private Map<K, T> buffer = Collections
+      .synchronizedMap(new LinkedHashMap<K, T>());
+
+  private DynamoDBStore<K, ? extends Persistent> dynamoDBStoreHandler;
+
+  /**
+   * Sets the handler to the main DynamoDB
+   * 
+   * @param DynamoDBStore
+   *          handler to main DynamoDB
+   */
+  @Override
+  public void setDynamoDBStoreHandler(DynamoDBStore<K, T> dynamoHandler) {
+    this.dynamoDBStoreHandler = dynamoHandler;
+  }
+
+  @Override
+  public void close() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void createSchema() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public boolean delete(K arg0) {
+    // TODO Auto-generated method stub
+    return false;
+  }
+
+  @Override
+  public long deleteByQuery(Query<K, T> arg0) {
+    // TODO Auto-generated method stub
+    return 0;
+  }
+
+  @Override
+  public void deleteSchema() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public Result<K, T> execute(Query<K, T> arg0) {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void flush() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public T get(K arg0, String[] arg1) {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> arg0)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public String getSchemaName() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public Query<K, T> newQuery() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void put(K key, T value) {
+    buffer.put(key, value);
+  }
+
+  @Override
+  public boolean schemaExists() {
+    // TODO Auto-generated method stub
+    return false;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBFactory.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBFactory.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBFactory.java
new file mode 100644
index 0000000..589367e
--- /dev/null
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBFactory.java
@@ -0,0 +1,49 @@
+/*
+ * 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 org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class DynamoDBFactory {
+
+  /** Helper to write useful information into the logs. */
+  public static final Logger LOG = LoggerFactory
+      .getLogger(DynamoDBFactory.class);
+
+  @SuppressWarnings("unchecked")
+  public static <K, T extends Persistent> IDynamoDB<K, T> buildDynamoDBStore(
+      DynamoDBUtils.DynamoDBType serType) {
+    final IDynamoDB<K, T> ds;
+    switch (serType) {
+      case DYNAMO:
+        ds = new DynamoDBNativeStore<K, T>();
+        LOG.debug("Using DynamoDB based serialization mode.");
+        break;
+      case AVRO:
+        ds = (IDynamoDB<K, T>) new DynamoDBAvroStore<K, PersistentBase>();
+        LOG.debug("Using Avro based serialization mode.");
+        break;
+      default:
+        throw new IllegalStateException("Serialization mode not supported.");
+    }
+    return ds;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBMapping.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBMapping.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBMapping.java
index f6b23a9..2db5fd8 100644
--- a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBMapping.java
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBMapping.java
@@ -18,90 +18,102 @@
 
 package org.apache.gora.dynamodb.store;
 
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.DYNAMO_KEY_HASHRANGE;
+
 import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.List;
 import java.util.Map;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.amazonaws.services.dynamodb.model.KeySchema;
-import com.amazonaws.services.dynamodb.model.KeySchemaElement;
-import com.amazonaws.services.dynamodb.model.ProvisionedThroughput;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeyType;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
 
 public class DynamoDBMapping {
-  
+
   /**
    * Helper to write useful information into the logs
    */
   public static final Logger LOG = LoggerFactory.getLogger(DynamoDBMapping.class);
-  
+
   /**
    *  a map from field name to attribute value
    */
-  private final Map<String, List<Map<String, String>>> tablesToItems;
-  
+  private final Map<String, Map<String, String>> tablesToItems;
+
   /**
    * Maps tables to their own key schemas
    */
-  private final Map<String, KeySchema> tablesToKeySchemas;
-  
+  private final Map<String, ArrayList<KeySchemaElement>> tablesToKeySchemas;
+
   /**
    * Maps tables to their provisioned throughput
    */
   private final Map<String, ProvisionedThroughput> tablesToPrTh;
-  
+
   /**
-   * Constructor for DynamoDBMapping 
-   * @param tables	Tables mapped.
-   * @param tablesToKeySchemas	KeySchemas used within tables mapped.
-   * @param provisionedThroughput	Provisioned throughput used within tables mapped.
+   * Constructor for DynamoDBMapping
+   * 
+   * @param tablesToItems2
+   *          Tables mapped.
+   * @param tablesToKeySchemas
+   *          KeySchemas used within tables mapped.
+   * @param provisionedThroughput
+   *          Provisioned throughput used within tables mapped.
    */
-  public DynamoDBMapping(Map<String, List<Map<String, String>>> tables,
-    Map<String, KeySchema> tablesToKeySchemas,
-    Map<String, ProvisionedThroughput> provisionedThroughput) {
-    
-    this.tablesToItems = tables;
+  public DynamoDBMapping(Map<String, Map<String, String>> tablesToItems2,
+      Map<String, ArrayList<KeySchemaElement>> tablesToKeySchemas,
+      Map<String, ProvisionedThroughput> provisionedThroughput) {
+
+    this.tablesToItems = tablesToItems2;
     this.tablesToKeySchemas = tablesToKeySchemas;
     this.tablesToPrTh = provisionedThroughput;
   }
 
   /**
    * Gets the tables with their own items
-   * @return tablesToItem HashMap 
+   * 
+   * @return tablesToItem 
+   *          HashMap 
    */
-  public Map<String,List<Map<String, String>>> getTables(){
+  public Map<String, Map<String, String>> getTables() {
     return tablesToItems;
   }
-  
+
   /**
    * Gets items or attributes from a specific table
-   * @param tableName	Table name to determine which attributes to get 
+   * 
+   * @param tableName
+   *          table name to determine which attributes to get 
    * @return
    */
-  public List<Map<String, String>> getItems(String tableName){
+  public Map<String, String> getItems(String tableName) {
     return tablesToItems.get(tableName);
   }
 
   /**
    * Gets the key schema from a specific table
-   * @param tableName	Table name to determine which key schema to get
+   * @param tableName
+   *          Table name to determine which key schema to get
    * @return
    */
-  public KeySchema getKeySchema(String tableName) {
+  public ArrayList<KeySchemaElement> getKeySchema(String tableName) {
     return tablesToKeySchemas.get(tableName);
   }
-  
+
   /**
    * Gets the provisioned throughput from a specific table
-   * @param tableName	Table name to determine which provisioned throughput to get
+   * 
+   * @param tableName
+   *          Table name to determine which provisioned throughput to get
    * @return
    */
   public ProvisionedThroughput getProvisionedThroughput(String tableName){
     return tablesToPrTh.get(tableName);
   }
-  
+
   /**
    * A builder for creating the mapper. This will allow building a thread safe
    * {@link DynamoDBMapping} using simple immutabilty.
@@ -110,199 +122,206 @@ public class DynamoDBMapping {
   public static class DynamoDBMappingBuilder {
 
     /**
-     * Table name to be used to build the DynamoDBMapping object 
-     */
-    private String tableName;
-  
-    /**
      * This data structure can hold several tables, with their own items.
      * Map<TableName, List<Map<AttributeName,AttributeType>>
      */
-    private Map<String, List<Map<String, String>>> tablesToItems = 
-      new HashMap<String, List<Map<String, String>>>();
+    private Map<String, Map<String, String>> tablesToItems = 
+        new HashMap<String, Map<String, String>>();
 
     /**
      * Maps tables to key schemas
      */
-    private Map<String, KeySchema> tablesToKeySchemas = new HashMap<String, KeySchema>();
+    private Map<String, ArrayList<KeySchemaElement>> tablesToKeySchemas = 
+        new HashMap<String, ArrayList<KeySchemaElement>>();
 
     /**
      * Maps tables to provisioned throughput
      */
-    private Map<String, ProvisionedThroughput> tablesToPrTh = new HashMap<String, ProvisionedThroughput>();
-  
-    /**
-     * Sets table name
-     * @param tabName
-     */
-    public void setTableName(String tabName){
-      tableName = tabName;
-    }
-  
+    private Map<String, ProvisionedThroughput> tablesToPrTh = 
+        new HashMap<String, ProvisionedThroughput>();
+
     /**
      * Gets the table name for which the table is being mapped
+     * 
      * @param tableName
      * @return
      */
     public String getTableName(String tableName){
       return tableName;
     }
-  
+
     /**
      * Sets the provisioned throughput for the specified table
+     * 
      * @param tableName
      * @param readCapUnits
      * @param writeCapUnits
      */
-    public void setProvisionedThroughput(String tableName, long readCapUnits, long writeCapUnits){
-      ProvisionedThroughput ptDesc = 
-      new ProvisionedThroughput().withReadCapacityUnits(readCapUnits).withWriteCapacityUnits(writeCapUnits);
-      tablesToPrTh.put(tableName, ptDesc);
+    public void setProvisionedThroughput(String tableName, long readCapUnits,
+        long writeCapUnits) {
+      ProvisionedThroughput ptDesc = new ProvisionedThroughput()
+          .withReadCapacityUnits(readCapUnits).withWriteCapacityUnits(
+              writeCapUnits);
     }
-  
+
     /**
      * Sets the hash range key schema for the specified table
+     * 
      * @param tableName
      * @param rangeKeyName
      * @param rangeKeyType
      */
-    public void setHashRangeKeySchema(String tableName, String rangeKeyName, String rangeKeyType){
-      KeySchema kSchema = tablesToKeySchemas.get(tableName);
-      if ( kSchema == null)
-        kSchema = new KeySchema();
-   
-      KeySchemaElement rangeKeyElement = new KeySchemaElement().withAttributeName(rangeKeyName).withAttributeType(rangeKeyType);
-      kSchema.setRangeKeyElement(rangeKeyElement);
-      tablesToKeySchemas.put(tableName, kSchema);
-    }
-  
+    // public void setHashRangeKeySchema(String tableName, String rangeKeyName,
+    // String rangeKeyType){
+    // KeySchemaElement kSchema = tablesToKeySchemas.get(tableName);
+    // if ( kSchema == null)
+    // kSchema = new KeySchemaElement();
+
+    // KeySchemaElement rangeKeyElement = new
+    // KeySchemaElement().withAttributeName(rangeKeyName).withKeyType(KeyType.RANGE).withKeyType(rangeKeyType);
+    // kSchema.
+    // kSchema.setRangeKeyElement(rangeKeyElement);
+    // tablesToKeySchemas.put(tableName, kSchema);
+    // }
+
     /**
      * Sets the hash key schema for the specified table
      * @param tableName
      * @param keyName
      * @param keyType
+     * @param keyType2
      */
-    public void setHashKeySchema(String tableName, String keyName, String keyType){
-      KeySchema kSchema = tablesToKeySchemas.get(tableName);
-        if ( kSchema == null)
-          kSchema = new KeySchema();
-        KeySchemaElement hashKey = new KeySchemaElement().withAttributeName(keyName).withAttributeType(keyType);
-        kSchema.setHashKeyElement(hashKey);
+    public void setKeySchema(String tableName, String keyName, String keyType) {
+      ArrayList<KeySchemaElement> kSchema = tablesToKeySchemas.get(tableName);
+      if (kSchema == null) {
+        kSchema = new ArrayList<KeySchemaElement>();
         tablesToKeySchemas.put(tableName, kSchema);
+      }
+      KeyType type = keyType.equals(DYNAMO_KEY_HASHRANGE) ? KeyType.RANGE : KeyType.HASH;
+      kSchema.add(new KeySchemaElement().withAttributeName(keyName)
+          .withKeyType(type));
     }
-  
+
     /**
-     * Checks if a table exists, and if doesn't exist it creates the new table. 
+     * Checks if a table exists, and if doesn't exist it creates the new table.
+     * 
      * @param tableName
      * @return The table identified by the parameter
      */
-    private List<Map<String, String>> getOrCreateTable(String tableName) {
-      
-      List<Map<String, String>> items = tablesToItems.get(tableName);
+    private Map<String, String> getOrCreateTable(String tableName) {
+      Map<String, String> items = tablesToItems.get(tableName);
       if (items == null) {
-        items = new ArrayList<Map<String, String>>();
+        items = new HashMap<String, String>();
         tablesToItems.put(tableName, items);
       }
       return items;
     }
-  
+
     /**
-     * Gets the attribute for a specific item. The idea is to be able to get different items with different attributes.
-     * TODO This method is incomplete because the itemNumber might not be present and this would be a problem
+     * Gets the attribute for a specific item. The idea is to be able to get 
+     * different items with different attributes.
+     * TODO This method is incomplete because the itemNumber might not 
+     * be present and this would be a problem
+     * 
      * @param items
      * @param itemNumber
      * @return
      */
-    private HashMap<String, String> getOrCreateItemAttribs(List<Map<String, String>> items, int itemNumber){
+    /*private HashMap<String, String> getOrCreateItemAttribs(
+        Map<String, String> items) {
       HashMap<String, String> itemAttribs;
-     
+
       if (items.isEmpty())
         items.add(new HashMap<String, String>());
-     
+
       itemAttribs = (HashMap<String, String>) items.get(itemNumber);
-      if (itemAttribs == null)
-        items.add(new HashMap<String, String>());
-        return (HashMap<String, String>) items.get(itemNumber);
-    }
-      
+      if (itemAttribs == null) {
+        itemAttribs = new HashMap<String, String>();
+      }
+
+      items.add(itemAttribs);
+      return null;
+    }*/
+
     /**
      * Adds an attribute to an specific item
+     * 
      * @param tableName
      * @param attributeName
      * @param attrType
      * @param itemNumber
      */
-     public void addAttribute(String tableName, String attributeName, String attrType, int itemNumber) {
-       // selecting table
-       List<Map<String, String>> items = getOrCreateTable(tableName);
-       // add attribute to item
-       HashMap<String, String> itemAttribs = getOrCreateItemAttribs(items, itemNumber);
-       itemAttribs.put(attributeName, attrType);
-       //items.add(itemAttribs);
-       // add item to table
-       //tablesToItems.put(tableName, items);
-     }
-  
+    public void addAttribute(String tableName, String attributeName,
+        String attrType) {
+      // selecting table
+      Map<String, String> items = getOrCreateTable(tableName);
+      // add attribute to item
+      //HashMap<String, String> itemAttribs = getOrCreateItemAttribs(items);
+      //itemAttribs.put(attributeName, attrType);
+      // add item to table
+      items.put(attributeName, attrType);
+      // tablesToItems.put(tableName, items);
+    }
+
     /**
      * Method to verify whether or not the schemas have been initialized
+     * 
      * @return
      */
-    private String verifyAllKeySchemas(){
-  
-      String wrongTable = "";
-      // if there are not tables defined
-      if (tablesToItems.isEmpty()) return "";
-        for(String tableName : tablesToItems.keySet()){
-          // if there are not schemas defined
-          if (tablesToKeySchemas.isEmpty()) return "";
-          if (!verifyKeySchema(tableName)) return "";
+    private boolean verifyAllKeySchemas() {
+      boolean rsl = true;
+      if (tablesToItems.isEmpty() || tablesToKeySchemas.isEmpty())
+        rsl = false;
+      for (String tableName : tablesToItems.keySet()) {
+        // if there are not schemas defined
+        if (tablesToKeySchemas.get(tableName) == null) {
+          LOG.error("No schema defined for DynamoDB table '" + tableName + '\'');
+          rsl = false;
         }
-      return wrongTable;
+        rsl = verifyKeySchema(tableName);
+      }
+      return rsl;
     }
-  
+
     /**
      * Verifies is a table has a key schema defined
+     * 
      * @param tableName	Table name to determine which key schema to obtain 
      * @return
      */
-    private boolean verifyKeySchema(String tableName){
-      KeySchema kSchema = tablesToKeySchemas.get(tableName);
-  
-      if (kSchema == null) 
-        return false;
-  
-      KeySchemaElement rangeKey = kSchema.getRangeKeyElement();
-      KeySchemaElement hashKey = kSchema.getHashKeyElement();
-      // A range key must have a hash key as well
-      
-      if (rangeKey != null){
-        if (hashKey != null)	
-          return true;
-        else 	  
-          return false;
+    private boolean verifyKeySchema(String tableName) {
+      ArrayList<KeySchemaElement> kSchema = tablesToKeySchemas.get(tableName);
+      boolean hashPk = false;
+      if (kSchema == null) {
+        LOG.error("No keys defined for '{}'. Please check your schema!", tableName);
+        return hashPk;
       }
-      // A hash key may exist by itself
-      if (hashKey != null)  
-        return true;
-      return false;
+      for (KeySchemaElement ks : kSchema) {
+        if (ks.getKeyType().equals(KeyType.HASH.toString())) {
+          hashPk = true;
+        }
+      }
+      return hashPk;
     }
-  
+
     /**
      * Constructs the DynamoDBMapping object
+     * 
      * @return A newly constructed mapping.
      */
     public DynamoDBMapping build() {
 
-      if (tableName == null) throw new IllegalStateException("tableName is not specified");
-        // verifying items for at least a table
-        if (tablesToItems.isEmpty()) throw new IllegalStateException("No tables");
-      
-        // verifying if key schemas have been properly defined
-        String wrongTableName = verifyAllKeySchemas();  
-        if (!wrongTableName.equals("")) throw new IllegalStateException("no key schemas defined for table " + wrongTableName);
-     
-        // Return the tableDescription and all the attributes needed
-            return new DynamoDBMapping(tablesToItems,tablesToKeySchemas, tablesToPrTh);
+      // verifying items for at least a table
+      if (tablesToItems.isEmpty())
+        throw new IllegalStateException("No tables were defined.");
+
+      // verifying if key schemas have been properly defined
+      if (!verifyAllKeySchemas())
+        throw new IllegalStateException("no key schemas defined for table ");
+
+      // Return the tableDescription and all the attributes needed
+      return new DynamoDBMapping(tablesToItems, tablesToKeySchemas,
+          tablesToPrTh);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBNativeStore.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBNativeStore.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBNativeStore.java
new file mode 100644
index 0000000..6fe1742
--- /dev/null
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBNativeStore.java
@@ -0,0 +1,552 @@
+/*
+ * 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 static org.apache.gora.dynamodb.store.DynamoDBUtils.WS_PROVIDER;
+
+import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Properties;
+
+import org.apache.commons.lang.NullArgumentException;
+import org.apache.gora.dynamodb.query.DynamoDBKey;
+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.store.ws.impl.WSDataStoreBase;
+import org.apache.gora.util.GoraException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBMapper;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBScanExpression;
+
+public class DynamoDBNativeStore<K, T extends Persistent> extends 
+    WSDataStoreBase<K, T> implements IDynamoDB<K, T> {
+
+  /** Method's names for getting range and hash keys. */
+  private static final String GET_RANGE_KEY_METHOD = "getRangeKey";
+  private static final String GET_HASH_KEY_METHOD = "getHashKey";
+
+  /** Logger for {@link DynamoDBNativeStore}. */
+  public static final Logger LOG = LoggerFactory
+      .getLogger(DynamoDBNativeStore.class);
+
+  /** Handler to {@link DynamoDBStore} so common methods can be accessed. */
+  private DynamoDBStore<K, T> dynamoDBStoreHandler;
+
+  /**
+   * Deletes items using a specific query
+   * 
+   * @throws IOException
+   */
+  @Override
+  @SuppressWarnings("unchecked")
+  public long deleteByQuery(Query<K, T> query) {
+    // TODO verify whether or not we are deleting a whole row
+    // String[] fields = getFieldsToQuery(query.getFields());
+    // find whether all fields are queried, which means that complete
+    // rows will be deleted
+    // boolean isAllFields = Arrays.equals(fields
+    // , getBeanFactory().getCachedPersistent().getFields());
+    Result<K, T> result = execute(query);
+    ArrayList<T> deletes = new ArrayList<T>();
+    try {
+      while (result.next()) {
+        T resultObj = result.get();
+        deletes.add(resultObj);
+
+        @SuppressWarnings("rawtypes")
+        DynamoDBKey dKey = new DynamoDBKey();
+
+        dKey.setHashKey(getHashFromObj(resultObj));
+
+        dKey.setRangeKey(getRangeKeyFromObj(resultObj));
+        delete((K) dKey);
+      }
+    } catch (IllegalArgumentException e) {
+      LOG.error("Illegal argument detected", e.getMessage());
+      throw new IllegalArgumentException(e);
+    } catch (IllegalAccessException e) {
+      LOG.error("Illegal access detected", e.getMessage());
+      throw new IllegalAccessError(e.getMessage());
+    } catch (InvocationTargetException e) {
+      LOG.error(e.getMessage());
+      throw new RuntimeException(e);
+    } catch (Exception e) {
+      LOG.error(e.getMessage());
+      throw new RuntimeException(e);
+    }
+    return deletes.size();
+  }
+
+  /**
+   * Executes a query after building a DynamoDB specific query based on the
+   * received one
+   */
+  @Override
+  public Result<K, T> execute(Query<K, T> query) {
+    DynamoDBQuery<K, T> dynamoDBQuery = buildDynamoDBQuery(query);
+    DynamoDBMapper mapper = new DynamoDBMapper(
+        dynamoDBStoreHandler.getDynamoDbClient());
+    List<T> objList = null;
+    if (DynamoDBQuery.getType().equals(DynamoDBQuery.RANGE_QUERY))
+      objList = mapper.scan(persistentClass,
+          (DynamoDBScanExpression) dynamoDBQuery.getQueryExpression());
+    if (DynamoDBQuery.getType().equals(DynamoDBQuery.SCAN_QUERY))
+      objList = mapper.scan(persistentClass,
+          (DynamoDBScanExpression) dynamoDBQuery.getQueryExpression());
+    return new DynamoDBResult<K, T>(this, query, objList);
+  }
+
+  @Override
+  public T get(K key, String[] fields) {
+    /*
+     * 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;
+     */
+    return null;
+  }
+
+  @Override
+  /**
+   * Gets the object with the specific key
+   * @throws IOException
+   */
+  public T get(K key) {
+    T object = null;
+    try {
+      Object rangeKey;
+      rangeKey = getRangeKeyFromKey(key);
+      Object hashKey = getHashFromKey(key);
+      if (hashKey != null) {
+        DynamoDBMapper mapper = new DynamoDBMapper(
+            dynamoDBStoreHandler.getDynamoDbClient());
+        if (rangeKey != null)
+          object = mapper.load(persistentClass, hashKey, rangeKey);
+        else
+          object = mapper.load(persistentClass, hashKey);
+      } else
+        throw new GoraException("Error while retrieving keys from object: "
+            + key.toString());
+    } catch (IllegalArgumentException e) {
+      LOG.error("Illegal argument detected", e.getMessage());
+      throw new IllegalArgumentException(e);
+    } catch (IllegalAccessException e) {
+      LOG.error("Illegal access detected", e.getMessage());
+      throw new IllegalAccessError(e.getMessage());
+    } catch (InvocationTargetException e) {
+      LOG.error(e.getMessage());
+      throw new RuntimeException(e);
+    } catch (GoraException ge) {
+      LOG.error(ge.getMessage());
+      LOG.error(ge.getStackTrace().toString());
+    }
+    return object;
+  }
+
+  /**
+   * Creates a new DynamoDBQuery
+   */
+  public Query<K, T> newQuery() {
+    Query<K, T> query = new DynamoDBQuery<K, T>(this);
+    // query.setFields(getFieldsToQuery(null));
+    return query;
+  }
+
+  /**
+   * Returns a new instance of the key object.
+   * 
+   * @throws IOException
+   */
+  @Override
+  public K newKey() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  /**
+   * Returns a new persistent object
+   * 
+   * @throws IOException
+   */
+  @Override
+  public T newPersistent() {
+    T obj = null;
+    try {
+      obj = persistentClass.newInstance();
+    } catch (InstantiationException e) {
+      LOG.error("Error instantiating " + persistentClass.getCanonicalName());
+      throw new InstantiationError(e.getMessage());
+    } catch (IllegalAccessException e) {
+      LOG.error("Error instantiating " + persistentClass.getCanonicalName());
+      throw new IllegalAccessError(e.getMessage());
+    }
+    return obj;
+  }
+
+  /**
+   * Puts an object identified by a key
+   * 
+   * @throws IOException
+   */
+  @Override
+  public void put(K key, T obj) {
+    try {
+      Object hashKey = getHashKey(key, obj);
+      Object rangeKey = getRangeKey(key, obj);
+      if (hashKey != null) {
+        DynamoDBMapper mapper = new DynamoDBMapper(
+            dynamoDBStoreHandler.getDynamoDbClient());
+        if (rangeKey != null) {
+          mapper.load(persistentClass, hashKey, rangeKey);
+        } else {
+          mapper.load(persistentClass, hashKey);
+        }
+        mapper.save(obj);
+      } else
+        throw new GoraException("No HashKey found in Key nor in Object.");
+    } catch (NullPointerException npe) {
+      LOG.error("Error while putting an item. " + npe.toString());
+      throw new NullArgumentException(npe.getMessage());
+    } catch (Exception e) {
+      LOG.error("Error while putting an item. " + obj.toString());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Deletes the object using key
+   * 
+   * @return true for a successful process
+   * @throws IOException
+   */
+  @Override
+  public boolean delete(K key) {
+    try {
+      T object = null;
+      Object rangeKey = null, hashKey = null;
+      DynamoDBMapper mapper = new DynamoDBMapper(
+          dynamoDBStoreHandler.getDynamoDbClient());
+      for (Method met : key.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_RANGE_KEY_METHOD)) {
+          Object[] params = null;
+          rangeKey = met.invoke(key, params);
+          break;
+        }
+      }
+      for (Method met : key.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_HASH_KEY_METHOD)) {
+          Object[] params = null;
+          hashKey = met.invoke(key, params);
+          break;
+        }
+      }
+      if (hashKey == null)
+        object = (T) mapper.load(persistentClass, key);
+      if (rangeKey == null)
+        object = (T) mapper.load(persistentClass, hashKey);
+      else
+        object = (T) mapper.load(persistentClass, hashKey, rangeKey);
+
+      if (object == null)
+        return false;
+
+      // setting key for dynamodbMapper
+      mapper.delete(object);
+      return true;
+    } catch (Exception e) {
+      LOG.error("Error while deleting value with key " + key.toString());
+      LOG.error(e.getMessage());
+      return false;
+    }
+  }
+
+  /**
+   * Initialize the data store by reading the credentials, setting the cloud
+   * provider, setting the client's properties up, setting the end point and
+   * reading the mapping file
+   */
+  public void initialize(Class<K> keyClass, Class<T> pPersistentClass,
+      Properties properties) {
+    super.initialize(keyClass, pPersistentClass, properties);
+    setWsProvider(WS_PROVIDER);
+    if (autoCreateSchema) {
+      createSchema();
+    }
+  }
+
+  /**
+   * Builds a DynamoDB query from a generic Query object
+   * 
+   * @param query
+   *          Generic query object
+   * @return DynamoDBQuery
+   */
+  private DynamoDBQuery<K, T> buildDynamoDBQuery(Query<K, T> query) {
+    if (getSchemaName() == null)
+      throw new IllegalStateException("There is not a preferred schema set.");
+
+    DynamoDBQuery<K, T> dynamoDBQuery = new DynamoDBQuery<K, T>();
+    dynamoDBQuery.setKeySchema(dynamoDBStoreHandler.getDynamoDbMapping()
+        .getKeySchema(getSchemaName()));
+    dynamoDBQuery.setKeyItems(dynamoDBStoreHandler.getDynamoDbMapping().getItems(getSchemaName()));
+    dynamoDBQuery.setQuery(query);
+    dynamoDBQuery.setConsistencyReadLevel(dynamoDBStoreHandler
+        .getConsistencyReads());
+    dynamoDBQuery.buildExpression();
+
+    return dynamoDBQuery;
+  }
+
+  @Override
+  public void close() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void flush() {
+    LOG.warn("DynamoDBNativeStore puts and gets directly into the datastore");
+  }
+
+  @Override
+  public BeanFactory<K, T> getBeanFactory() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> arg0)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override
+  public void setBeanFactory(BeanFactory<K, T> arg0) {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public void createSchema() {
+    LOG.info("Creating Native DynamoDB Schemas.");
+    if (dynamoDBStoreHandler.getDynamoDbMapping().getTables().isEmpty()) {
+      throw new IllegalStateException("There are not tables defined.");
+    }
+    if (dynamoDBStoreHandler.getPreferredSchema() == null) {
+      LOG.debug("Creating schemas.");
+      // read the mapping object
+      for (String tableName : dynamoDBStoreHandler.getDynamoDbMapping()
+          .getTables().keySet())
+        DynamoDBUtils.executeCreateTableRequest(
+            dynamoDBStoreHandler.getDynamoDbClient(), tableName,
+            dynamoDBStoreHandler.getTableKeySchema(tableName),
+            dynamoDBStoreHandler.getTableAttributes(tableName),
+            dynamoDBStoreHandler.getTableProvisionedThroughput(tableName));
+      LOG.debug("tables created successfully.");
+    } else {
+      String tableName = dynamoDBStoreHandler.getPreferredSchema();
+      LOG.debug("Creating schema " + tableName);
+      DynamoDBUtils.executeCreateTableRequest(
+          dynamoDBStoreHandler.getDynamoDbClient(), tableName,
+          dynamoDBStoreHandler.getTableKeySchema(tableName),
+          dynamoDBStoreHandler.getTableAttributes(tableName),
+          dynamoDBStoreHandler.getTableProvisionedThroughput(tableName));
+    }
+  }
+
+  /*
+   * (non-Javadoc)
+   * 
+   * @see
+   * org.apache.gora.dynamodb.store.IDynamoDB#setDynamoDBStoreHandler(org.apache
+   * .gora.dynamodb.store.DynamoDBStore)
+   */
+  @Override
+  public void setDynamoDBStoreHandler(DynamoDBStore<K, T> dynamoHandler) {
+    this.dynamoDBStoreHandler = dynamoHandler;
+  }
+
+  @Override
+  public void deleteSchema() {
+    // TODO Auto-generated method stub
+
+  }
+
+  @Override
+  public String getSchemaName() {
+    return this.dynamoDBStoreHandler.getSchemaName();
+  }
+
+  @Override
+  public boolean schemaExists() {
+    return this.dynamoDBStoreHandler.schemaExists();
+  }
+
+  private Object getHashKey(K key, T obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    // try to get the hashKey from 'key'
+    Object hashKey = getHashFromKey(key);
+    // if the key does not have these attributes then try to get them from the
+    // object
+    if (hashKey == null)
+      hashKey = getHashFromObj(obj);
+    // if no key has been found, then we try with the key
+    if (hashKey == null)
+      hashKey = key;
+    return hashKey;
+  }
+
+  /**
+   * Gets a hash key from a key of type K
+   * 
+   * @param obj
+   *          Object from which we will get a hash key
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   * @throws InvocationTargetException
+   */
+  private Object getHashFromKey(K obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    Object hashKey = null;
+    // check if it is a DynamoDBKey
+    if (obj instanceof DynamoDBKey) {
+      hashKey = ((DynamoDBKey<?, ?>) obj).getHashKey();
+    } else {
+      // maybe the class has the method defined
+      for (Method met : obj.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_HASH_KEY_METHOD)) {
+          Object[] params = null;
+          hashKey = met.invoke(obj, params);
+          break;
+        }
+      }
+    }
+    return hashKey;
+  }
+
+  /**
+   * Gets a hash key from an object of type T
+   * 
+   * @param obj
+   *          Object from which we will get a hash key
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   * @throws InvocationTargetException
+   */
+  private Object getHashFromObj(T obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    Object hashKey = null;
+    // check if it is a DynamoDBKey
+    if (obj instanceof DynamoDBKey) {
+      hashKey = ((DynamoDBKey<?, ?>) obj).getHashKey();
+    } else {
+      // maybe the class has the method defined
+      for (Method met : obj.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_HASH_KEY_METHOD)) {
+          Object[] params = null;
+          hashKey = met.invoke(obj, params);
+          break;
+        }
+      }
+    }
+    return hashKey;
+  }
+
+  private Object getRangeKey(K key, T obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    Object rangeKey = getRangeKeyFromKey(key);
+    if (rangeKey == null)
+      rangeKey = getRangeKeyFromObj(obj);
+    return rangeKey;
+  }
+
+  /**
+   * Gets a range key from a key obj. This verifies if it is using a
+   * {@link DynamoDBKey}
+   * 
+   * @param obj
+   *          Object from which a range key will be extracted
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   * @throws InvocationTargetException
+   */
+  private Object getRangeKeyFromKey(K obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    Object rangeKey = null;
+    // check if it is a DynamoDBKey
+    if (obj instanceof DynamoDBKey) {
+      rangeKey = ((DynamoDBKey<?, ?>) obj).getRangeKey();
+    } else {
+      // maybe the class has the method defined
+      for (Method met : obj.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_RANGE_KEY_METHOD)) {
+          Object[] params = null;
+          rangeKey = met.invoke(obj, params);
+          break;
+        }
+      }
+    }
+    return rangeKey;
+  }
+
+  /**
+   * Gets a range key from an object T
+   * 
+   * @param obj
+   *          Object from which a range key will be extracted
+   * @return
+   * @throws IllegalArgumentException
+   * @throws IllegalAccessException
+   * @throws InvocationTargetException
+   */
+  private Object getRangeKeyFromObj(T obj) throws IllegalArgumentException,
+  IllegalAccessException, InvocationTargetException {
+    Object rangeKey = null;
+    // check if it is a DynamoDBKey
+    if (obj instanceof DynamoDBKey) {
+      rangeKey = ((DynamoDBKey<?, ?>) obj).getRangeKey();
+    } else {
+      // maybe the class has the method defined
+      for (Method met : obj.getClass().getDeclaredMethods()) {
+        if (met.getName().equals(GET_RANGE_KEY_METHOD)) {
+          Object[] params = null;
+          rangeKey = met.invoke(obj, params);
+          break;
+        }
+      }
+    }
+    return rangeKey;
+  }
+
+}
\ No newline at end of file


[5/5] gora git commit: Merge branch 'GORA-362' this closes #56

Posted by le...@apache.org.
Merge branch 'GORA-362' this closes #56


Project: http://git-wip-us.apache.org/repos/asf/gora/repo
Commit: http://git-wip-us.apache.org/repos/asf/gora/commit/96e623b0
Tree: http://git-wip-us.apache.org/repos/asf/gora/tree/96e623b0
Diff: http://git-wip-us.apache.org/repos/asf/gora/diff/96e623b0

Branch: refs/heads/master
Commit: 96e623b035b2ab35fca15d03b790bee55636efcd
Parents: d46a96f 655cd3a
Author: Lewis John McGibbney <le...@jpl.nasa.gov>
Authored: Wed Mar 16 07:41:38 2016 -0700
Committer: Lewis John McGibbney <le...@jpl.nasa.gov>
Committed: Wed Mar 16 07:41:38 2016 -0700

----------------------------------------------------------------------
 conf/log4j.properties                           |    2 +-
 .../gora/cassandra/store/CassandraStore.java    |    4 +-
 .../apache/gora/mapreduce/GoraRecordReader.java |   80 +-
 .../gora/mapreduce/PersistentDeserializer.java  |   14 +-
 .../gora/mapreduce/PersistentSerialization.java |   11 +-
 .../gora/mapreduce/PersistentSerializer.java    |    8 +-
 .../org/apache/gora/memory/store/MemStore.java  |    3 +-
 .../org/apache/gora/persistency/Persistent.java |    5 +-
 .../org/apache/gora/persistency/Tombstone.java  |   17 +
 .../gora/store/ws/impl/WSDataStoreBase.java     |   10 +-
 .../java/org/apache/gora/util/AvroUtils.java    |   10 +-
 .../org/apache/gora/util/ReflectionUtils.java   |   47 +-
 gora-dynamodb/pom.xml                           |  254 +++--
 gora-dynamodb/src/examples/avro/person.json     |   13 +
 .../gora/dynamodb/example/generated/Person.java |   79 ++
 .../dynamodb/example/generated/Webpage.java     |   72 ++
 .../apache/gora/examples/generated/person.java  |   97 --
 .../dynamodb/compiler/GoraDynamoDBCompiler.java |  399 +++----
 .../apache/gora/dynamodb/query/DynamoDBKey.java |   27 +-
 .../gora/dynamodb/query/DynamoDBQuery.java      |  327 ++++--
 .../gora/dynamodb/store/DynamoDBAvroStore.java  |  136 +++
 .../gora/dynamodb/store/DynamoDBFactory.java    |   49 +
 .../gora/dynamodb/store/DynamoDBMapping.java    |  297 ++---
 .../dynamodb/store/DynamoDBNativeStore.java     |  552 ++++++++++
 .../gora/dynamodb/store/DynamoDBStore.java      | 1036 +++++++-----------
 .../gora/dynamodb/store/DynamoDBUtils.java      |  229 ++++
 .../apache/gora/dynamodb/store/IDynamoDB.java   |   31 +
 .../src/test/conf/gora-dynamodb-mapping.xml     |   10 +-
 gora-dynamodb/src/test/conf/gora.properties     |    3 +
 .../gora/dynamodb/GoraDynamoDBTestDriver.java   |   89 +-
 .../gora/dynamodb/TestDynamoDBNativeStore.java  |  409 +++++++
 .../apache/gora/dynamodb/TestDynamoDBStore.java |  386 -------
 .../apache/gora/mongodb/store/MongoStore.java   |    3 +-
 pom.xml                                         |    4 +-
 34 files changed, 2862 insertions(+), 1851 deletions(-)
----------------------------------------------------------------------



[4/5] gora git commit: GORA-362 Refactor gora-dynamodb to support avro serialization

Posted by le...@apache.org.
GORA-362 Refactor gora-dynamodb to support avro serialization


Project: http://git-wip-us.apache.org/repos/asf/gora/repo
Commit: http://git-wip-us.apache.org/repos/asf/gora/commit/655cd3aa
Tree: http://git-wip-us.apache.org/repos/asf/gora/tree/655cd3aa
Diff: http://git-wip-us.apache.org/repos/asf/gora/diff/655cd3aa

Branch: refs/heads/master
Commit: 655cd3aa846751f4805ef5e84ecd9193fe2c7aa2
Parents: 4f98503
Author: Lewis John McGibbney <le...@jpl.nasa.gov>
Authored: Fri Feb 26 17:48:49 2016 -0800
Committer: Lewis John McGibbney <le...@jpl.nasa.gov>
Committed: Fri Feb 26 17:48:49 2016 -0800

----------------------------------------------------------------------
 conf/log4j.properties                           |    2 +-
 .../gora/cassandra/store/CassandraStore.java    |    4 +-
 .../apache/gora/mapreduce/GoraRecordReader.java |   80 +-
 .../gora/mapreduce/PersistentDeserializer.java  |   14 +-
 .../gora/mapreduce/PersistentSerialization.java |   11 +-
 .../gora/mapreduce/PersistentSerializer.java    |    8 +-
 .../org/apache/gora/memory/store/MemStore.java  |    3 +-
 .../org/apache/gora/persistency/Persistent.java |    5 +-
 .../org/apache/gora/persistency/Tombstone.java  |   17 +
 .../gora/store/ws/impl/WSDataStoreBase.java     |   10 +-
 .../java/org/apache/gora/util/AvroUtils.java    |   10 +-
 .../org/apache/gora/util/ReflectionUtils.java   |   47 +-
 gora-dynamodb/pom.xml                           |  254 +++--
 gora-dynamodb/src/examples/avro/person.json     |   13 +
 .../gora/dynamodb/example/generated/Person.java |   79 ++
 .../dynamodb/example/generated/Webpage.java     |   72 ++
 .../apache/gora/examples/generated/person.java  |   97 --
 .../dynamodb/compiler/GoraDynamoDBCompiler.java |  399 +++----
 .../apache/gora/dynamodb/query/DynamoDBKey.java |   27 +-
 .../gora/dynamodb/query/DynamoDBQuery.java      |  327 ++++--
 .../gora/dynamodb/store/DynamoDBAvroStore.java  |  136 +++
 .../gora/dynamodb/store/DynamoDBFactory.java    |   49 +
 .../gora/dynamodb/store/DynamoDBMapping.java    |  297 ++---
 .../dynamodb/store/DynamoDBNativeStore.java     |  552 ++++++++++
 .../gora/dynamodb/store/DynamoDBStore.java      | 1036 +++++++-----------
 .../gora/dynamodb/store/DynamoDBUtils.java      |  229 ++++
 .../apache/gora/dynamodb/store/IDynamoDB.java   |   31 +
 .../src/test/conf/gora-dynamodb-mapping.xml     |   10 +-
 gora-dynamodb/src/test/conf/gora.properties     |    3 +
 .../gora/dynamodb/GoraDynamoDBTestDriver.java   |   89 +-
 .../gora/dynamodb/TestDynamoDBNativeStore.java  |  409 +++++++
 .../apache/gora/dynamodb/TestDynamoDBStore.java |  386 -------
 .../apache/gora/mongodb/store/MongoStore.java   |    3 +-
 pom.xml                                         |    4 +-
 34 files changed, 2862 insertions(+), 1851 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/conf/log4j.properties
----------------------------------------------------------------------
diff --git a/conf/log4j.properties b/conf/log4j.properties
index 8293138..745d358 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -26,7 +26,7 @@ log4j.rootLogger=${gora.root.logger}
 #special logging requirements for some commandline tools
 log4j.logger.org.apache.gora.compiler.GoraCompiler=INFO,console
 log4j.logger.org.apache.avro.specific.SpecificCompiler=INFO,console
-log4j.logger.org.apache.gora.dynamodb.compiler.GoraDynamoDBCompiler=INFO,console
+log4j.logger.org.apache.gora.dynamodb.compiler.GoraDynamoDBCompiler=DEBUG,console
 log4j.logger.org.apache.gora.tutorial.log.LogManager=INFO,console
 log4j.logger.org.apache.gora.tutorial.log.LogAnalytics=INFO,console
 

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
----------------------------------------------------------------------
diff --git a/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java b/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
index bbed362..2619129 100644
--- a/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
+++ b/gora-cassandra/src/main/java/org/apache/gora/cassandra/store/CassandraStore.java
@@ -442,8 +442,8 @@ public class CassandraStore<K, T extends PersistentBase> extends DataStoreBase<K
   private Object getFieldValue(Schema fieldSchema, Type type, Object fieldValue ){
     switch(type) {
     case RECORD:
-      Persistent persistent = (Persistent) fieldValue;
-      Persistent newRecord = (Persistent) SpecificData.get().newRecord(persistent, persistent.getSchema());
+      PersistentBase persistent = (PersistentBase) fieldValue;
+      PersistentBase newRecord = (PersistentBase) SpecificData.get().newRecord(persistent, persistent.getSchema());
       for (Field member: fieldSchema.getFields()) {
         if (member.pos() == 0 || !persistent.isDirty()) {
           continue;

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/mapreduce/GoraRecordReader.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/mapreduce/GoraRecordReader.java b/gora-core/src/main/java/org/apache/gora/mapreduce/GoraRecordReader.java
index 2460246..832afc6 100644
--- a/gora-core/src/main/java/org/apache/gora/mapreduce/GoraRecordReader.java
+++ b/gora-core/src/main/java/org/apache/gora/mapreduce/GoraRecordReader.java
@@ -42,31 +42,31 @@ public class GoraRecordReader<K, T extends PersistentBase> extends RecordReader<
 
   protected Query<K,T> query;
   protected Result<K,T> result;
-  
+
   private GoraRecordCounter counter = new GoraRecordCounter();
-  
+
   public GoraRecordReader(Query<K,T> query, TaskAttemptContext context) {
     this.query = query;
 
     Configuration configuration = context.getConfiguration();
     int recordsMax = configuration.getInt(BUFFER_LIMIT_READ_NAME, BUFFER_LIMIT_READ_VALUE);
-    
+
     // Check if result set will at least contain 2 rows
     if (recordsMax <= 1) {
       LOG.info("Limit " + recordsMax + " changed to " + BUFFER_LIMIT_READ_VALUE);
       recordsMax = BUFFER_LIMIT_READ_VALUE;
     }
-    
+
     counter.setRecordsMax(recordsMax);
     LOG.info("gora.buffer.read.limit = " + recordsMax);
-    
+
     this.query.setLimit(recordsMax);
   }
 
   public void executeQuery() throws Exception {
     this.result = query.execute();
   }
-  
+
   @Override
   public K getCurrentKey() throws IOException, InterruptedException {
     return result.getKey();
@@ -80,47 +80,47 @@ public class GoraRecordReader<K, T extends PersistentBase> extends RecordReader<
   @Override
   public float getProgress() throws IOException, InterruptedException {
     try{
-	  return result.getProgress();
-  	}
- 	catch(Exception e){
- 		return 0;
- 	}
+      return result.getProgress();
+    }
+    catch(Exception e){
+      return 0;
+    }
   }
 
   @Override
   public void initialize(InputSplit split, TaskAttemptContext context)
-  throws IOException, InterruptedException { }
+      throws IOException, InterruptedException { }
 
   @Override
   public boolean nextKeyValue() throws IOException, InterruptedException {
-	  try{
-	    if (counter.isModulo()) {
-	      boolean firstBatch = (this.result == null);
-	      if (! firstBatch) {
-	        this.query.setStartKey(this.result.getKey());
-	        if (this.query.getLimit() == counter.getRecordsMax()) {
-	          this.query.setLimit(counter.getRecordsMax() + 1);
-	        }
-	      }
-	      if (this.result != null) {
-	        this.result.close();
-	      }
-	      
-	      executeQuery();
-	      
-	      if (! firstBatch) {
-	        // skip first result
-	        this.result.next();
-	      }
-	    }
-	    
-	    counter.increment();
-	    return this.result.next();
-	  }
-	  catch(Exception e){
-	    LOG.error("Error reading Gora records: {}", e.getMessage());
-	    throw new RuntimeException(e);
-	  }
+    try{
+      if (counter.isModulo()) {
+        boolean firstBatch = (this.result == null);
+        if (! firstBatch) {
+          this.query.setStartKey(this.result.getKey());
+          if (this.query.getLimit() == counter.getRecordsMax()) {
+            this.query.setLimit(counter.getRecordsMax() + 1);
+          }
+        }
+        if (this.result != null) {
+          this.result.close();
+        }
+
+        executeQuery();
+
+        if (! firstBatch) {
+          // skip first result
+          this.result.next();
+        }
+      }
+
+      counter.increment();
+      return this.result.next();
+    }
+    catch(Exception e){
+      LOG.error("Error reading Gora records: {}", e.getMessage());
+      throw new RuntimeException(e);
+    }
   }
 
   //@Override

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentDeserializer.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentDeserializer.java b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentDeserializer.java
index d592872..79eee42 100644
--- a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentDeserializer.java
+++ b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentDeserializer.java
@@ -24,7 +24,7 @@ import org.apache.avro.Schema;
 import org.apache.avro.io.BinaryDecoder;
 import org.apache.avro.io.DecoderFactory;
 import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.util.AvroUtils;
 import org.apache.hadoop.io.serializer.Deserializer;
 
@@ -33,19 +33,19 @@ import org.apache.hadoop.io.serializer.Deserializer;
 * with {@link BinaryDecoder}.
 */
 public class PersistentDeserializer
-   implements Deserializer<Persistent> {
+   implements Deserializer<PersistentBase> {
 
   private BinaryDecoder decoder;
-  private Class<? extends Persistent> persistentClass;
+  private Class<? extends PersistentBase> persistentClass;
   private boolean reuseObjects;
-  private SpecificDatumReader<Persistent> datumReader;
+  private SpecificDatumReader<PersistentBase> datumReader;
 
-  public PersistentDeserializer(Class<? extends Persistent> c, boolean reuseObjects) {
+  public PersistentDeserializer(Class<? extends PersistentBase> c, boolean reuseObjects) {
     this.persistentClass = c;
     this.reuseObjects = reuseObjects;
     try {
       Schema schema = AvroUtils.getSchema(persistentClass);
-      datumReader = new SpecificDatumReader<>(schema);
+      datumReader = new SpecificDatumReader<PersistentBase>(schema);
 
     } catch (Exception ex) {
       throw new RuntimeException(ex);
@@ -67,7 +67,7 @@ public class PersistentDeserializer
   public void close() throws IOException { }
 
   @Override
-  public Persistent deserialize(Persistent persistent) throws IOException {
+  public PersistentBase deserialize(PersistentBase persistent) throws IOException {
     return datumReader.read(reuseObjects ? persistent : null, decoder);
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerialization.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerialization.java b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerialization.java
index 008e222..6ee1f4e 100644
--- a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerialization.java
+++ b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerialization.java
@@ -17,26 +17,25 @@
  */
 package org.apache.gora.mapreduce;
 
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.Serialization;
 import org.apache.hadoop.io.serializer.Serializer;
 
-public class PersistentSerialization
-implements Serialization<Persistent> {
+public class PersistentSerialization implements Serialization<PersistentBase> {
 
   @Override
   public boolean accept(Class<?> c) {
-    return Persistent.class.isAssignableFrom(c);
+    return PersistentBase.class.isAssignableFrom(c);
   }
 
   @Override
-  public Deserializer<Persistent> getDeserializer(Class<Persistent> c) {
+  public Deserializer<PersistentBase> getDeserializer(Class<PersistentBase> c) {
     return new PersistentDeserializer(c, true);
   }
 
   @Override
-  public Serializer<Persistent> getSerializer(Class<Persistent> c) {
+  public Serializer<PersistentBase> getSerializer(Class<PersistentBase> c) {
     return new PersistentSerializer();
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerializer.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerializer.java b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerializer.java
index 4b1a646..b63de09 100644
--- a/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerializer.java
+++ b/gora-core/src/main/java/org/apache/gora/mapreduce/PersistentSerializer.java
@@ -23,16 +23,16 @@ import java.io.OutputStream;
 import org.apache.avro.io.BinaryEncoder;
 import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.hadoop.io.serializer.Serializer;
 
 /**
  * Hadoop serializer using Avro's {@link SpecificDatumWriter}
  * with {@link BinaryEncoder}.
  */
-public class PersistentSerializer implements Serializer<Persistent> {
+public class PersistentSerializer implements Serializer<PersistentBase> {
 
-  private SpecificDatumWriter<Persistent> datumWriter;
+  private SpecificDatumWriter<PersistentBase> datumWriter;
   private BinaryEncoder encoder;
   
   public PersistentSerializer() {
@@ -58,7 +58,7 @@ public class PersistentSerializer implements Serializer<Persistent> {
    * Do the serialization of the {@link Persistent} object
    */
   @Override
-  public void serialize(Persistent persistent) throws IOException {
+  public void serialize(PersistentBase persistent) throws IOException {
     datumWriter.setSchema(persistent.getSchema());
         
     datumWriter.write(persistent, encoder);

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java b/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
index e5f9aca..81dc3b9 100644
--- a/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
+++ b/gora-core/src/main/java/org/apache/gora/memory/store/MemStore.java
@@ -28,7 +28,6 @@ import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.avro.Schema.Field;
-import org.apache.gora.persistency.Persistent;
 import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.PartitionQuery;
 import org.apache.gora.query.Query;
@@ -170,7 +169,7 @@ public class MemStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
   /**
    * Returns a clone with exactly the requested fields shallowly copied
    */
-  private static<T extends Persistent> T getPersistent(T obj, String[] fields) {
+  private static<T extends PersistentBase> T getPersistent(T obj, String[] fields) {
     List<Field> otherFields = obj.getSchema().getFields();
     String[] otherFieldStrings = new String[otherFields.size()];
     for(int i = 0; i<otherFields.size(); i++ ){

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java b/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java
index dc45f79..bd520a9 100644
--- a/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java
+++ b/gora-core/src/main/java/org/apache/gora/persistency/Persistent.java
@@ -20,14 +20,11 @@ package org.apache.gora.persistency;
 import java.util.List;
 
 import org.apache.avro.Schema.Field;
-import org.apache.avro.specific.SpecificRecord;
-
-import org.apache.gora.persistency.Dirtyable;
 
 /**
 * Objects that are persisted by Gora implements this interface.
 */
-public interface Persistent extends SpecificRecord, Dirtyable {
+public interface Persistent extends Dirtyable {
 
   public static String DIRTY_BYTES_FIELD_NAME = "__g__dirty";
 

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java b/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
index 61d85e9..527fd4e 100644
--- a/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
+++ b/gora-core/src/main/java/org/apache/gora/persistency/Tombstone.java
@@ -1,3 +1,20 @@
+/*
+* 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.persistency;
 
 /**

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSDataStoreBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSDataStoreBase.java b/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSDataStoreBase.java
index e1d907b..494a09d 100644
--- a/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSDataStoreBase.java
+++ b/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSDataStoreBase.java
@@ -22,6 +22,7 @@ import java.util.Properties;
 
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
+import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.util.StringUtils;
 
 /**
@@ -54,6 +55,11 @@ implements DataStore<K, T>{
    * Properties object 
    */
   protected Properties properties;
+  
+  /** 
+   * Determines if an schema will be automatically created. 
+   */
+  protected boolean autoCreateSchema;
 
   /**
    * Default constructor
@@ -61,14 +67,16 @@ implements DataStore<K, T>{
   public WSDataStoreBase() {
   }
 
-  @Override
   /**
    * Initializes the web services backed data store
    */
+  @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
       Properties properties) {
     setKeyClass(keyClass);
     setPersistentClass(persistentClass);
+    autoCreateSchema = DataStoreFactory.getAutoCreateSchema(properties, this);
+    this.properties = properties;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/util/AvroUtils.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/util/AvroUtils.java b/gora-core/src/main/java/org/apache/gora/util/AvroUtils.java
index 86ab4a3..36b076e 100644
--- a/gora-core/src/main/java/org/apache/gora/util/AvroUtils.java
+++ b/gora-core/src/main/java/org/apache/gora/util/AvroUtils.java
@@ -33,7 +33,7 @@ import org.apache.avro.io.EncoderFactory;
 import org.apache.avro.reflect.ReflectData;
 import org.apache.avro.specific.SpecificDatumReader;
 import org.apache.avro.specific.SpecificDatumWriter;
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 
 /**
  * An utility class for Avro related tasks
@@ -65,7 +65,7 @@ public class AvroUtils {
   /**
    * Returns the schema of the class
    */
-  public static Schema getSchema(Class<? extends Persistent> clazz)
+  public static Schema getSchema(Class<? extends PersistentBase> clazz)
       throws SecurityException, NoSuchFieldException, IllegalArgumentException,
       IllegalAccessException {
 
@@ -80,7 +80,7 @@ public class AvroUtils {
    *          the persistent object to get the fields names from
    * @return the field names
    */
-  public static String[] getPersistentFieldNames(Persistent persistent) {
+  public static String[] getPersistentFieldNames(PersistentBase persistent) {
     return getSchemaFieldNames(persistent.getSchema());
   }
 
@@ -100,10 +100,10 @@ public class AvroUtils {
     return fieldNames;
   }
 
-  public static <T extends Persistent> T deepClonePersistent(T persistent) {
+  public static <T extends PersistentBase> T deepClonePersistent(T persistent) {
     ByteArrayOutputStream bos = new ByteArrayOutputStream();
     BinaryEncoder enc = EncoderFactory.get().binaryEncoder(bos, null);
-    SpecificDatumWriter<Persistent> writer = new SpecificDatumWriter<>(
+    SpecificDatumWriter<PersistentBase> writer = new SpecificDatumWriter<>(
         persistent.getSchema());
     try {
       writer.write(persistent, enc);

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-core/src/main/java/org/apache/gora/util/ReflectionUtils.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/util/ReflectionUtils.java b/gora-core/src/main/java/org/apache/gora/util/ReflectionUtils.java
index a43dd64..bf3dc32 100644
--- a/gora-core/src/main/java/org/apache/gora/util/ReflectionUtils.java
+++ b/gora-core/src/main/java/org/apache/gora/util/ReflectionUtils.java
@@ -21,7 +21,7 @@ package org.apache.gora.util;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import org.apache.avro.specific.SpecificRecordBuilderBase;
-import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.impl.PersistentBase;
 
 /**
  * Utility methods related to reflection
@@ -30,12 +30,12 @@ public class ReflectionUtils {
 
   public static Class<?>[] EMPTY_CLASS_ARRAY = new Class<?>[0];
   public static Object[] EMPTY_OBJECT_ARRAY = new Object[0];
-  
+
   /**
    * Returns the empty argument constructor of the class.
    */
   public static<T> Constructor<T> getConstructor(Class<T> clazz) 
-    throws SecurityException, NoSuchMethodException {
+      throws SecurityException, NoSuchMethodException {
     if(clazz == null) {
       throw new IllegalArgumentException("class cannot be null");
     }
@@ -43,12 +43,12 @@ public class ReflectionUtils {
     cons.setAccessible(true);
     return cons;
   }
-  
+
   /**
    * Returns whether the class defines an empty argument constructor.
    */
   public static boolean hasConstructor(Class<?> clazz) 
-  throws SecurityException, NoSuchMethodException {
+      throws SecurityException, NoSuchMethodException {
     if(clazz == null) {
       throw new IllegalArgumentException("class cannot be null");
     }
@@ -69,44 +69,47 @@ public class ReflectionUtils {
    * @param clazz the class of the object
    * @return a new instance of the object
    */
-  public static <T> T newInstance(Class<T> clazz) throws InstantiationException
-  , IllegalAccessException, SecurityException, NoSuchMethodException
-  , IllegalArgumentException, InvocationTargetException {
-    
+  public static <T> T newInstance(Class<T> clazz) 
+      throws InstantiationException, IllegalAccessException, 
+      SecurityException, NoSuchMethodException, IllegalArgumentException, 
+      InvocationTargetException {
+
     Constructor<T> cons = getConstructor(clazz);
-    
+
     return cons.newInstance(EMPTY_OBJECT_ARRAY);
   }
-  
+
   /**
    * Constructs a new instance of the class using the no-arg constructor.
    * @param classStr the class name of the object
    * @return a new instance of the object
    */
-  public static Object newInstance(String classStr) throws InstantiationException
-    , IllegalAccessException, ClassNotFoundException, SecurityException
-    , IllegalArgumentException, NoSuchMethodException, InvocationTargetException {
+  public static Object newInstance(String classStr) 
+      throws InstantiationException, IllegalAccessException, 
+      ClassNotFoundException, SecurityException, IllegalArgumentException, 
+      NoSuchMethodException, InvocationTargetException {
     if(classStr == null) {
       throw new IllegalArgumentException("class cannot be null");
     }
     Class<?> clazz = ClassLoadingUtils.loadClass(classStr);
     return newInstance(clazz);
   }
-  
+
   /**
    * Returns the value of a named static field
    */
   public static Object getStaticField(Class<?> clazz, String fieldName) 
-  throws IllegalArgumentException, SecurityException,
-  IllegalAccessException, NoSuchFieldException {
-    
+      throws IllegalArgumentException, SecurityException,
+      IllegalAccessException, NoSuchFieldException {
+
     return clazz.getField(fieldName).get(null);
   }
-  
+
   @SuppressWarnings("unchecked")
-  public static <T extends Persistent> SpecificRecordBuilderBase<T> classBuilder(Class<T> clazz) throws SecurityException
-    , NoSuchMethodException, IllegalArgumentException, IllegalAccessException, InvocationTargetException {
+  public static <T extends PersistentBase> SpecificRecordBuilderBase<T> classBuilder(Class<T> clazz) 
+      throws SecurityException, NoSuchMethodException, IllegalArgumentException, 
+      IllegalAccessException, InvocationTargetException {
     return (SpecificRecordBuilderBase<T>) clazz.getMethod("newBuilder").invoke(null);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/pom.xml
----------------------------------------------------------------------
diff --git a/gora-dynamodb/pom.xml b/gora-dynamodb/pom.xml
index a234c8f..c2356cb 100644
--- a/gora-dynamodb/pom.xml
+++ b/gora-dynamodb/pom.xml
@@ -14,137 +14,131 @@
    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.6.1-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 
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.gora</groupId>
+    <artifactId>gora</artifactId>
+    <version>0.7-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>
-       <testResources>
-            <testResource>
-                <directory>src/test/conf/</directory>
-                <includes>
-                    <include>**</include>
-                </includes>
-            </testResource>
-       </testResources>
-       <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-surefire-plugin</artifactId>
-               <version>${maven-surfire-plugin.version}</version>
-               <configuration>
-                   <skipTests>true</skipTests>
-               </configuration>
-           </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>
-           <type>test-jar</type>
-           <scope>test</scope>
-       </dependency>
-       
-       <!-- Amazon Dependencies -->
-       <dependency>
-           <groupId>com.amazonaws</groupId>
-	   <artifactId>aws-java-sdk</artifactId>
-       </dependency>
-       
-       <!-- Misc Dependencies -->
-       <dependency>
-            <groupId>org.jdom</groupId>
-            <artifactId>jdom</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>
+  <inceptionYear>2010</inceptionYear>
+  <organization>
+    <name>The Apache Software Foundation</name>
+    <url>http://www.apache.org/</url>
+  </organization>
+  <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>
+    <testResources>
+      <testResource>
+        <directory>src/test/conf/</directory>
+        <includes>
+          <include>**</include>
+        </includes>
+      </testResource>
+    </testResources>
+    <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-surefire-plugin</artifactId>
+        <configuration>
+          <skipTests>true</skipTests>
+        </configuration>
+      </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>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+
+    <!-- Amazon Dependencies -->
+    <dependency>
+      <groupId>com.amazonaws</groupId>
+      <artifactId>aws-java-sdk</artifactId>
+    </dependency>
+
+    <!-- Misc Dependencies -->
+    <dependency>
+      <groupId>org.jdom</groupId>
+      <artifactId>jdom</artifactId>
+    </dependency>
+
+    <!-- Logging Dependencies -->
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <!-- Testing Dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+  </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/examples/avro/person.json
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/examples/avro/person.json b/gora-dynamodb/src/examples/avro/person.json
new file mode 100644
index 0000000..b8c4ca2
--- /dev/null
+++ b/gora-dynamodb/src/examples/avro/person.json
@@ -0,0 +1,13 @@
+{
+  "type": "record",
+  "name": "person",
+  "namespace": "org.apache.gora.examples.dynamodb.generated",
+  "fields" : [
+    {"name": "ssn", "type": "string"},
+    {"name": "date", "type": "string"}, 
+    {"name": "firstname", "type": "string"},
+    {"name": "lastname", "type": "string"},
+    {"name": "lastname", "type": "integer"}, 
+    {"name": "visitedplaces", "type": {"type":"array", "items": "string"}}
+  ]
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Person.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Person.java b/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Person.java
new file mode 100644
index 0000000..471fb59
--- /dev/null
+++ b/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Person.java
@@ -0,0 +1,79 @@
+package org.apache.gora.dynamodb.example.generated;
+
+import java.util.List;
+import java.util.Set;
+
+import org.apache.avro.Schema.Field;
+import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.Tombstone;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBAttribute;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBHashKey;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBRangeKey;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBTable;
+
+@DynamoDBTable(tableName = "Person")
+public class Person implements Persistent {
+    private double ssn;
+
+    @DynamoDBHashKey(attributeName="ssn") 
+    public double getHashKey() {  return ssn; } 
+    public void setHashKey(double pSsn){  this.ssn = pSsn; }
+
+    private String date;
+
+    @DynamoDBRangeKey(attributeName="date") 
+    public String getRangeKey() { return date; } 
+    public void setRangeKey(String pDate){  this.date = pDate; }
+
+    private String lastName;
+    @DynamoDBAttribute(attributeName = "LastName")
+    public String getLastName() {  return lastName;  }
+    public void setLastName(String pLastName) {  this.lastName = pLastName;  }
+
+    private Set<String> visitedplaces;
+    @DynamoDBAttribute(attributeName = "Visitedplaces")
+    public Set<String> getVisitedplaces() {  return visitedplaces;  }
+    public void setVisitedplaces(Set<String> pVisitedplaces) {  this.visitedplaces = pVisitedplaces;  }
+
+    private double salary;
+    @DynamoDBAttribute(attributeName = "Salary")
+    public double getSalary() {  return salary;  }
+    public void setSalary(double pSalary) {  this.salary = pSalary;  }
+
+    private String firstName;
+    @DynamoDBAttribute(attributeName = "FirstName")
+    public String getFirstName() {  return firstName;  }
+    public void setFirstName(String pFirstName) {  this.firstName = pFirstName;  }
+
+
+    public void setNew(boolean pNew){}
+    public void setDirty(boolean pDirty){}
+    @Override
+    public void clear() { }
+    @Override
+    public Person clone() { return null; }
+    @Override
+    public boolean isDirty() { return false; }
+    @Override
+    public boolean isDirty(int fieldIndex) { return false; }
+    @Override
+    public boolean isDirty(String field) { return false; }
+    @Override
+    public void setDirty() { }
+    @Override
+    public void setDirty(int fieldIndex) { }
+    @Override
+    public void setDirty(String field) { }
+    @Override
+    public void clearDirty(int fieldIndex) { }
+    @Override
+    public void clearDirty(String field) { }
+    @Override
+    public void clearDirty() { }
+    @Override
+    public Tombstone getTombstone() { return null; }
+    @Override
+    public List<Field> getUnmanagedFields() { return null; }
+    @Override
+    public Persistent newInstance() { return new Person(); }
+}

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Webpage.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Webpage.java b/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Webpage.java
new file mode 100644
index 0000000..df9eb80
--- /dev/null
+++ b/gora-dynamodb/src/examples/java/org/apache/gora/dynamodb/example/generated/Webpage.java
@@ -0,0 +1,72 @@
+package org.apache.gora.dynamodb.example.generated;
+
+import java.util.List;
+
+import org.apache.avro.Schema.Field;
+import org.apache.gora.persistency.Persistent;
+import org.apache.gora.persistency.Tombstone;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBAttribute;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBHashKey;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBRangeKey;
+import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBTable;
+
+@DynamoDBTable(tableName = "Webpage")
+public class Webpage implements Persistent {
+    private String id;
+
+    @DynamoDBHashKey(attributeName="id") 
+    public String getHashKey() {  return id; } 
+    public void setHashKey(String pId){  this.id = pId; }
+
+    private String content;
+    @DynamoDBAttribute(attributeName = "Content")
+    public String getContent() {  return content;  }
+    public void setContent(String pContent) {  this.content = pContent;  }
+
+    private String common;
+    @DynamoDBAttribute(attributeName = "Common")
+    public String getCommon() {  return common;  }
+    public void setCommon(String pCommon) {  this.common = pCommon;  }
+
+    private String outlinks;
+    @DynamoDBAttribute(attributeName = "Outlinks")
+    public String getOutlinks() {  return outlinks;  }
+    public void setOutlinks(String pOutlinks) {  this.outlinks = pOutlinks;  }
+
+    private String parsedContent;
+    @DynamoDBAttribute(attributeName = "ParsedContent")
+    public String getParsedContent() {  return parsedContent;  }
+    public void setParsedContent(String pParsedContent) {  this.parsedContent = pParsedContent;  }
+
+
+    public void setNew(boolean pNew){}
+    public void setDirty(boolean pDirty){}
+    @Override
+    public void clear() { }
+    @Override
+    public Webpage clone() { return null; }
+    @Override
+    public boolean isDirty() { return false; }
+    @Override
+    public boolean isDirty(int fieldIndex) { return false; }
+    @Override
+    public boolean isDirty(String field) { return false; }
+    @Override
+    public void setDirty() { }
+    @Override
+    public void setDirty(int fieldIndex) { }
+    @Override
+    public void setDirty(String field) { }
+    @Override
+    public void clearDirty(int fieldIndex) { }
+    @Override
+    public void clearDirty(String field) { }
+    @Override
+    public void clearDirty() { }
+    @Override
+    public Tombstone getTombstone() { return null; }
+    @Override
+    public List<Field> getUnmanagedFields() { return null; }
+    @Override
+    public Persistent newInstance() { return new Webpage(); }
+}

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/examples/java/org/apache/gora/examples/generated/person.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/examples/java/org/apache/gora/examples/generated/person.java b/gora-dynamodb/src/examples/java/org/apache/gora/examples/generated/person.java
deleted file mode 100644
index 66f940b..0000000
--- a/gora-dynamodb/src/examples/java/org/apache/gora/examples/generated/person.java
+++ /dev/null
@@ -1,97 +0,0 @@
-package org.apache.gora.examples.generated;
-import java.util.Set;
-import org.apache.gora.persistency.Persistent;
-import org.apache.gora.persistency.StateManager;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBAttribute;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBHashKey;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBRangeKey;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBTable;
-
-@DynamoDBTable(tableName = "person")
-public class person implements Persistent {
-    private String ssn;
-    private String date;
-
-    @DynamoDBHashKey(attributeName="ssn") 
-    public String getHashKey() {  return ssn; } 
-    public void setHashKey(String pSsn){  this.ssn = pSsn; }
-    @DynamoDBRangeKey(attributeName="date") 
-    public String getRangeKey() {  return date; } 
-    public void setRangeKey(String pDate){  this.date = pDate; }
-
-    private String lastName;
-    @DynamoDBAttribute(attributeName = "LastName")
-    public String getLastName() {  return lastName;  }
-    public void setLastName(String pLastName) {  this.lastName = pLastName;  }
-
-    private Set<String> visitedplaces;
-    @DynamoDBAttribute(attributeName = "Visitedplaces")
-    public Set<String> getVisitedplaces() {  return visitedplaces;  }
-    public void setVisitedplaces(Set<String> pVisitedplaces) {  this.visitedplaces = pVisitedplaces;  }
-
-    private double salary;
-    @DynamoDBAttribute(attributeName = "Salary")
-    public double getSalary() {  return salary;  }
-    public void setSalary(double pSalary) {  this.salary = pSalary;  }
-
-    private String firstName;
-    @DynamoDBAttribute(attributeName = "FirstName")
-    public String getFirstName() {  return firstName;  }
-    public void setFirstName(String pFirstName) {  this.firstName = pFirstName;  }
-
-
-    public void setNew(boolean pNew){}
-    public void setDirty(boolean pDirty){}
-    @Override
-    public StateManager getStateManager() { return null; }
-    @Override
-    public Persistent newInstance(StateManager stateManager) { return null; }
-    @Override
-    public String[] getFields() { return null; }
-    @Override
-    public String getField(int index) { return null; }
-    @Override
-    public int getFieldIndex(String field) { return 0; }
-    @Override
-    public void clear() { }
-    @Override
-    public person clone() { return null; }
-    @Override
-    public boolean isNew() { return false; }
-    @Override
-    public void setNew() { }
-    @Override
-    public void clearNew() { }
-    @Override
-    public boolean isDirty() { return false; }
-    @Override
-    public boolean isDirty(int fieldIndex) { return false; }
-    @Override
-    public boolean isDirty(String field) { return false; }
-    @Override
-    public void setDirty() { }
-    @Override
-    public void setDirty(int fieldIndex) { }
-    @Override
-    public void setDirty(String field) { }
-    @Override
-    public void clearDirty(int fieldIndex) { }
-    @Override
-    public void clearDirty(String field) { }
-    @Override
-    public void clearDirty() { }
-    @Override
-    public boolean isReadable(int fieldIndex) { return false; }
-    @Override
-    public boolean isReadable(String field) { return false; }
-    @Override
-    public void setReadable(int fieldIndex) { }
-    @Override
-    public void setReadable(String field) { }
-    @Override
-    public void clearReadable(int fieldIndex) { }
-    @Override
-    public void clearReadable(String field) { }
-    @Override
-    public void clearReadable() { }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/compiler/GoraDynamoDBCompiler.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/compiler/GoraDynamoDBCompiler.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/compiler/GoraDynamoDBCompiler.java
index cf03b33..be38e36 100644
--- a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/compiler/GoraDynamoDBCompiler.java
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/compiler/GoraDynamoDBCompiler.java
@@ -18,207 +18,241 @@
 
 package org.apache.gora.dynamodb.compiler;
 
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.MAPPING_FILE;
+
 import java.io.File;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.OutputStreamWriter;
 import java.io.Writer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.util.Map;
 
+import org.apache.commons.io.FilenameUtils;
 import org.apache.gora.dynamodb.store.DynamoDBMapping;
 import org.apache.gora.dynamodb.store.DynamoDBMapping.DynamoDBMappingBuilder;
+import org.apache.gora.util.GoraException;
 import org.jdom.Document;
 import org.jdom.Element;
 import org.jdom.input.SAXBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.amazonaws.services.dynamodb.model.KeySchema;
-import com.amazonaws.services.dynamodb.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.KeyType;
 
-/** Generate specific Java classes for defined schemas. */
+/** 
+ * Generate specific Java classes for defined schemas. 
+ * Different from the {@link org.apache.gora.compiler.GoraCompiler}, 
+ * which uses an .avsc or .json schema definition, this compiler 
+ * expects an XML schema file as input.
+ */
 public class GoraDynamoDBCompiler {
   private File dest;
   private Writer out;
   private static final Logger log = LoggerFactory.getLogger(GoraDynamoDBCompiler.class);
+  private String packageName;
 
+  /**
+   * GoraDynamoDBCompiler
+   * 
+   * @param File
+   *          where the data bean will be written.
+   */
   private GoraDynamoDBCompiler(File dest) {
-    this.dest = dest;                             // root directory for output
+    this.dest = dest;
   }
 
   /** Generates Java classes for a schema. */
   public static void compileSchema(File src, File dest) throws IOException {
-    log.info("Compiling " + src + " to " + dest );
+    log.info("Compiling {} to {}", src, dest);
     GoraDynamoDBCompiler compiler = new GoraDynamoDBCompiler(dest);
     DynamoDBMapping dynamoDBMap = compiler.readMapping(src);
-    if (dynamoDBMap.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
+    if (dynamoDBMap.getTables().isEmpty()) 
+      throw new IllegalStateException("There are no tables defined.");
 
-    for(String tableName : dynamoDBMap.getTables().keySet()){
-      compiler.compile(tableName, dynamoDBMap.getKeySchema(tableName), dynamoDBMap.getItems(tableName));
+    for(String tableName : dynamoDBMap.getTables().keySet()) {
+      compiler.compile(tableName, dynamoDBMap.getKeySchema(tableName), 
+          dynamoDBMap.getItems(tableName));
+      log.info("{} written without issues to {}", tableName, dest.getAbsolutePath());
     }
   }
 
   /**
-   * Method in charge of compiling a specific table using a key schema and a set of attributes
-   * @param pTableNameTable name
-   * @param pKeySchemaKey schema used
-   * @param pItemsList of items belonging to a specific table
+   * Method in charge of compiling a specific table using a key schema and a set 
+   * of attributes
+   * @param dest2 
+   * @param pTableNameTable 
+   *          name
+   * @param pKeySchemaKey 
+   *          schema used
+   * @param pItemsList 
+   *          of items belonging to a specific table
    */
-  private void compile(String pTableName, KeySchema pKeySchema, List<Map<String, String>> pItems){
-    // TODO define where the generated will go 
+  private void compile(String pTableName, ArrayList<KeySchemaElement> arrayList, Map<String, String> map){
     try {
       startFile(pTableName, pTableName);
-      setHeaders(null);
+      setHeaders(packageName);
       line(0, "");
       line(0, "@DynamoDBTable(tableName = \"" + pTableName + "\")");
       line(0, "public class " + pTableName + " implements Persistent {");
-      setKeyAttributes(pKeySchema, 2);
-      setKeyMethods(pKeySchema, 2);
-      setItems(pItems, 2);
-      setDefaultMethods(2);
+      for (KeySchemaElement pKeySchema : arrayList) {
+        setKeyAttributes(pKeySchema, map.get(pKeySchema.getAttributeName()), 2);
+        setKeyMethods(pKeySchema, map.get(pKeySchema.getAttributeName()), 2);
+        map.remove(pKeySchema.getAttributeName());
+      }
+      setItems(map, 2);
+      setDefaultMethods(2, pTableName);
       line(0, "}");
       out.flush();
       out.close();
     } catch (IOException e) {
-      log.error("Error while compiling table " + pTableName);
-      e.printStackTrace();
+      log.error("Error while compiling table {}",pTableName, e.getMessage());
+      throw new RuntimeException(e);
     }
   }
-  
+
   /**
    * Receives a list of all items and creates getters and setters for them
-   * @param pItemsThe items belonging to the table
-   * @param pIdenThe number of spaces used for identation
+   * @param pItemsThe 
+   *          items belonging to the table
+   * @param pIdenThe 
+   *          number of spaces used for identation
    * @throws IOException
    */
-  private void setItems(List<Map<String, String>> pItems, int pIden) throws IOException{
-    for(Map<String, String> item : pItems){
-      for (String itemName : item.keySet()){
-        String itemType = "String";
-        if (item.get(itemName).toString().equals("N"))
-          itemType = "double";
-        if (item.get(itemName).toString().equals("SS"))
-          itemType = "Set<String>";
-        if (item.get(itemName).toString().equals("SN"))
-          itemType = "Set<double>";
-        line(pIden, "private " + itemType + " " + itemName + ";");
-        setItemMethods(itemName, itemType, pIden);
-      }
+  private void setItems(Map<String, String> pItems, int pIden)
+      throws IOException {
+    for (String itemName : pItems.keySet()) {
+      String itemType = "String";
+      if (pItems.get(itemName).toString().equals("N"))
+        itemType = "double";
+      if (pItems.get(itemName).toString().equals("SS"))
+        itemType = "Set<String>";
+      if (pItems.get(itemName).toString().equals("SN"))
+        itemType = "Set<double>";
+      line(pIden, "private " + itemType + " " + itemName + ";");
+      setItemMethods(itemName, itemType, pIden);
     }
     line(0, "");
   }
-  
+
   /**
    * Creates item getters and setters
-   * @param pItemNameItem's name
-   * @param pItemTypeItem's type
-   * @param pIdenNumber of spaces used for indentation
+   * @param pItemNameItem
+   *          's name
+   * @param pItemTypeItem
+   *          's type
+   * @param pIdenNumber 
+   *          of spaces used for indentation
    * @throws IOException
    */
-  private void setItemMethods(String pItemName, String pItemType, int pIden) throws IOException{
-    line(pIden, "@DynamoDBAttribute(attributeName = \"" + camelCasify(pItemName) + "\")");
-    line(pIden, "public " + pItemType + " get" + camelCasify(pItemName) + "() {  return " + pItemName + ";  }");
-    line(pIden, "public void set" + camelCasify(pItemName) + "(" + pItemType + " p" + camelCasify(pItemName) + ") {  this." + pItemName + " = p"+ camelCasify(pItemName) +";  }");
+  private void setItemMethods(String pItemName, String pItemType, int pIden)
+      throws IOException {
+    line(pIden, "@DynamoDBAttribute(attributeName = \""
+        + camelCasify(pItemName) + "\")");
+    line(pIden, "public " + pItemType + " get" + camelCasify(pItemName)
+    + "() {  return " + pItemName + ";  }");
+    line(pIden, "public void set" + camelCasify(pItemName) + "(" + pItemType
+        + " p" + camelCasify(pItemName) + ") {  this." + pItemName + " = p"
+        + camelCasify(pItemName) + ";  }");
     line(0, "");
   }
-  
+
   /**
    * Creates key getters and setters 
-   * @param pKeySchemaThe key schema for a specific table
-   * @param pIdenNumber of spaces used for indentation
+   * @param pKeySchemaThe 
+   *          key schema for a specific table
+   * @param pIdenNumber 
+   *          of spaces used for indentation
    * @throws IOException
    */
-  private void setKeyMethods(KeySchema pKeySchema, int pIden) throws IOException{
-    KeySchemaElement hashKey = pKeySchema.getHashKeyElement();
-    KeySchemaElement rangeKey = pKeySchema.getRangeKeyElement();
+  private void setKeyMethods(KeySchemaElement pKeySchema, String attType,
+      int pIden) throws IOException {
     StringBuilder strBuilder = new StringBuilder();
+    attType = attType.equals("S") ? "String" : "double";
     // hash key
-    if(hashKey != null){
-      strBuilder.append("@DynamoDBHashKey(attributeName=\"" + hashKey.getAttributeName() + "\") \n");
-      strBuilder.append("    public String getHashKey() {  return " + hashKey.getAttributeName() + "; } \n");
-      strBuilder.append("    public void setHashKey(" + (hashKey.getAttributeType().equals("S")?"String ":"double "));
-      strBuilder.append("p" + camelCasify(hashKey.getAttributeName()) + "){  this." + hashKey.getAttributeName());
-      strBuilder.append(" = p" + camelCasify(hashKey.getAttributeName()) + "; }");
+    if (pKeySchema.getKeyType().equals(KeyType.HASH.toString())) {
+      strBuilder.append("@DynamoDBHashKey(attributeName=\""
+          + pKeySchema.getAttributeName() + "\") \n");
+      strBuilder.append("    public " + attType + " getHashKey() {  return "
+          + pKeySchema.getAttributeName() + "; } \n");
+      strBuilder.append("    public void setHashKey(" + attType + " ");
+      strBuilder.append("p" + camelCasify(pKeySchema.getAttributeName())
+      + "){  this." + pKeySchema.getAttributeName());
+      strBuilder.append(" = p" + camelCasify(pKeySchema.getAttributeName())
+      + "; }");
       line(pIden, strBuilder.toString());
     }
     strBuilder.delete(0, strBuilder.length());
     // range key
-    if(rangeKey != null){
-      strBuilder.append("@DynamoDBRangeKey(attributeName=\"" + rangeKey.getAttributeName() + "\") \n");
-      strBuilder.append("    public String getRangeKey() {  return " + rangeKey.getAttributeName() + "; } \n");
-      strBuilder.append("    public void setRangeKey(" + (rangeKey.getAttributeType().equals("S")?"String ":"double "));
-      strBuilder.append("p" + camelCasify(rangeKey.getAttributeName()) + "){  this." + rangeKey.getAttributeName());
-      strBuilder.append(" = p" + camelCasify(rangeKey.getAttributeName()) + "; }");
+    if (pKeySchema.getKeyType().equals(KeyType.RANGE.toString())) {
+      strBuilder.append("@DynamoDBRangeKey(attributeName=\""
+          + pKeySchema.getAttributeName() + "\") \n");
+      strBuilder.append("    public " + attType + " getRangeKey() { return "
+          + pKeySchema.getAttributeName() + "; } \n");
+      strBuilder.append("    public void setRangeKey(" + attType + " ");
+      strBuilder.append("p" + camelCasify(pKeySchema.getAttributeName())
+      + "){  this." + pKeySchema.getAttributeName());
+      strBuilder.append(" = p" + camelCasify(pKeySchema.getAttributeName())
+      + "; }");
       line(pIden, strBuilder.toString());
     }
     line(0, "");
   }
-  
+
   /**
    * Creates the key attributes within the generated class
-   * @param pKeySchemaKey schema
-   * @param pIdenNumber of spaces used for indentation
+   * @param pKeySchema
+   *          schema
+   * @param attType
+   *          attribute type
+   * @param pIden
+   *          of spaces used for indentation
    * @throws IOException
    */
-  private void setKeyAttributes(KeySchema pKeySchema, int pIden) throws IOException{
-    KeySchemaElement hashKey = pKeySchema.getHashKeyElement();
-    KeySchemaElement rangeKey = pKeySchema.getRangeKeyElement();
+  private void setKeyAttributes(KeySchemaElement pKeySchema, String attType,
+      int pIden) throws IOException {
     StringBuilder strBuilder = new StringBuilder();
-    // hash key
-    if(hashKey != null){
-      strBuilder.append("private " + (hashKey.getAttributeType().equals("S")?"String ":"double "));
-      strBuilder.append(hashKey.getAttributeName() + ";");
+    attType = attType.equals("S") ? "String " : "double ";
+    if (pKeySchema != null) {
+      strBuilder.append("private " + attType);
+      strBuilder.append(pKeySchema.getAttributeName() + ";");
       line(pIden, strBuilder.toString());
     }
     strBuilder.delete(0, strBuilder.length());
-    // range key
-    if(rangeKey != null){
-      strBuilder.append("private " + (rangeKey.getAttributeType().equals("S")?"String ":"double "));
-      strBuilder.append(rangeKey.getAttributeName() + ";");
-      line(pIden, strBuilder.toString());
-    }
     line(0, "");
   }
-  
+
   /**
    * Returns camel case version of a string
-   * @param sString to be camelcasified
+   * @param sString 
+   *          to be camelcasified
    * @return
    */
   private static String camelCasify(String s) {
-    return s.substring(0, 1).toUpperCase() + s.substring(1);
-  }
-
-  /** Recognizes camel case */
-  private static String toUpperCase(String s) {
-    StringBuilder builder = new StringBuilder();
-    for(int i=0; i<s.length(); i++) {
-      if(i > 0) {
-        if(Character.isUpperCase(s.charAt(i))
-         && Character.isLowerCase(s.charAt(i-1))
-         && Character.isLetter(s.charAt(i))) {
-            builder.append("_");
-        }
-      }
-      builder.append(Character.toUpperCase(s.charAt(i)));
-    }
-    return builder.toString();
+    return s.substring(0, 1).toUpperCase(Locale.getDefault()) + s.substring(1);
   }
 
   /**
    * Starts the java generated class file
-   * @param nameClass name
+   * @param nameClass 
+   *          name
    * @param space
+   *          spacing
    * @throws IOException
    */
   private void startFile(String name, String space) throws IOException {
-    File dir = new File(dest, space.replace('.', File.separatorChar));
+    String fullDest = FilenameUtils.normalize
+        (dest.getAbsolutePath() + File.separatorChar + packageName.replace('.', File.separatorChar));
+    File dir = new File(fullDest);
     if (!dir.exists())
       if (!dir.mkdirs())
         throw new IOException("Unable to create " + dir);
     name = cap(name) + ".java";
-    out = new OutputStreamWriter(new FileOutputStream(new File(dir, name)));
+    out = new OutputStreamWriter(new FileOutputStream(new File(dir, name)), Charset.defaultCharset());
 
   }
 
@@ -228,46 +262,36 @@ public class GoraDynamoDBCompiler {
    * @throws IOException
    */
   private void setHeaders(String namespace) throws IOException {
-    if(namespace != null) {
-      line(0, "package "+namespace+";\n");
+    if (namespace != null) {
+      line(0, "package " + namespace + ";\n");
     }
+    line(0, "import java.util.List;");
     line(0, "import java.util.Set;");
+    line(0, "");
+    line(0, "import org.apache.avro.Schema.Field;");
     line(0, "import org.apache.gora.persistency.Persistent;");
-    line(0, "import org.apache.gora.persistency.StateManager;");
-    line(0, "import com.amazonaws.services.dynamodb.datamodeling.DynamoDBAttribute;");
-    line(0, "import com.amazonaws.services.dynamodb.datamodeling.DynamoDBHashKey;");
-    line(0, "import com.amazonaws.services.dynamodb.datamodeling.DynamoDBRangeKey;");
-    line(0, "import com.amazonaws.services.dynamodb.datamodeling.DynamoDBTable;");
+    line(0, "import org.apache.gora.persistency.Tombstone;");
+    line(0, "import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBAttribute;");
+    line(0, "import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBHashKey;");
+    line(0, "import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBRangeKey;");
+    line(0, "import com.amazonaws.services.dynamodbv2.datamodeling.DynamoDBTable;");
   }
 
   /**
    * Creates default methods inherited from upper classes
-   * @param pIdenNumber of spaces used for indentation
+   * @param pIden 
+   *          of spaces used for indentation
+   * @param tabName
+   *          table name
    * @throws IOException
    */
-  private void setDefaultMethods(int pIden) throws IOException {
+  private void setDefaultMethods(int pIden, String tabName) throws IOException {
     line(pIden, "public void setNew(boolean pNew){}");
     line(pIden, "public void setDirty(boolean pDirty){}");
     line(pIden, "@Override");
-    line(pIden, "public StateManager getStateManager() { return null; }");
-    line(pIden, "@Override");
-    line(pIden, "public Persistent newInstance(StateManager stateManager) { return null; }");
-    line(pIden, "@Override");
-    line(pIden, "public String[] getFields() { return null; }");
-    line(pIden, "@Override");
-    line(pIden, "public String getField(int index) {return null; }");
-    line(pIden, "@Override");
-    line(pIden, "public int getFieldIndex(String field) { return 0; }");
-    line(pIden, "@Override");
     line(pIden, "public void clear() { }");
     line(pIden, "@Override");
-    line(pIden, "public person clone() {return null; }");
-    line(pIden, "@Override");
-    line(pIden, "public boolean isNew() { return false; }");
-    line(pIden, "@Override");
-    line(pIden, "public void setNew() { }");
-    line(pIden, "@Override");
-    line(pIden, "public void clearNew() {}");
+    line(pIden, "public " + tabName + " clone() { return null; }");
     line(pIden, "@Override");
     line(pIden, "public boolean isDirty() { return false; }");
     line(pIden, "@Override");
@@ -287,29 +311,23 @@ public class GoraDynamoDBCompiler {
     line(pIden, "@Override");
     line(pIden, "public void clearDirty() { }");
     line(pIden, "@Override");
-    line(pIden, "public boolean isReadable(int fieldIndex) {return false; }");
-    line(pIden, "@Override");
-    line(pIden, "public boolean isReadable(String field) { return false; }");
-    line(pIden, "@Override");
-    line(pIden, "public void setReadable(int fieldIndex) { }");
-    line(pIden, "@Override");
-    line(pIden, "public void setReadable(String field) { }");
-    line(pIden, "@Override");
-    line(pIden, "public void clearReadable(int fieldIndex) { }");
+    line(pIden, "public Tombstone getTombstone() { return null; }");
     line(pIden, "@Override");
-    line(pIden, "public void clearReadable(String field) { }");
+    line(pIden, "public List<Field> getUnmanagedFields() { return null; }");
     line(pIden, "@Override");
-    line(pIden, "public void clearReadable() { }");
+    line(pIden, "public Persistent newInstance() { return new " + tabName + "(); }");
   }
 
   /**
    * Writes a line within the output stream
-   * @param indentNumber of spaces used for indentation
-   * @param textText to be written
+   * @param indentNumber 
+   *          of spaces used for indentation
+   * @param textText 
+   *          to be written
    * @throws IOException
    */
   private void line(int indent, String text) throws IOException {
-    for (int i = 0; i < indent; i ++) {
+    for (int i = 0; i < indent; i++) {
       out.append("  ");
     }
     out.append(text);
@@ -318,29 +336,38 @@ public class GoraDynamoDBCompiler {
 
   /**
    * Returns the string received with the first letter in uppercase
-   * @param nameString to be converted
+   * @param name 
+   *          to be converted
    * @return
    */
   static String cap(String name) {
-    return name.substring(0,1).toUpperCase()+name.substring(1,name.length());
+    return name.substring(0,1).toUpperCase(Locale.getDefault())
+        + name.substring(1,name.length());
   }
 
   /**
    * Start point of the compiler program
-   * @param argsReceives the schema file to be compiled and where this should be written
+   * @param argsReceives 
+   *          the schema file to be compiled and where this should be written
    * @throws Exception
    */
-  public static void main(String[] args) throws Exception {
-    if (args.length < 2) {
-      System.err.println("Usage: Compiler <schema file> <output dir>");
-      System.exit(1);
+  public static void main(String[] args) {
+    try {
+      if (args.length < 2) {
+        log.error("Usage: Compiler <schema file> <output dir>");
+        System.exit(1);
+      }
+      compileSchema(new File(args[0]), new File(args[1]));
+    } catch (Exception e) {
+      log.error("Something went wrong. Please check the input file.", e.getMessage());
+      throw new RuntimeException(e);
     }
-    compileSchema(new File(args[0]), new File(args[1]));
   }
 
   /**
    * Reads the schema file and converts it into a data structure to be used
-   * @param pMapFileThe schema file to be mapped into a table
+   * @param pMapFile
+   *          schema file to be mapped into a table
    * @return
    * @throws IOException
    */
@@ -352,50 +379,50 @@ public class GoraDynamoDBCompiler {
     try {
       SAXBuilder builder = new SAXBuilder();
       Document doc = builder.build(pMapFile);
-      
+
+      if (doc == null || doc.getRootElement() == null)
+        throw new GoraException("Unable to load " + MAPPING_FILE
+            + ". Please check its existance!");
+
       Element root = doc.getRootElement();
 
       List<Element> tableElements = root.getChildren("table");
-      for(Element tableElement : tableElements) {
-      
-      String tableName = tableElement.getAttributeValue("name");
-      long readCapacUnits = Long.parseLong(tableElement.getAttributeValue("readcunit"));
-      long writeCapacUnits = Long.parseLong(tableElement.getAttributeValue("readcunit"));
-    
-      mappingBuilder.setTableName(tableName);
-      mappingBuilder.setProvisionedThroughput(tableName, readCapacUnits, writeCapacUnits);
-      log.debug("Basic table properties have been set: Name, and Provisioned throughput.");
-    
-      // Retrieving key's features
-      List<Element> fieldElements = tableElement.getChildren("key");
-      for(Element fieldElement : fieldElements) {
-        String keyName  = fieldElement.getAttributeValue("name");
-        String keyType  = fieldElement.getAttributeValue("type");
-        String keyAttrType  = fieldElement.getAttributeValue("att-type");
-        if(keyType.equals("hash"))
-          mappingBuilder.setHashKeySchema(tableName, keyName, keyAttrType);
-        else if(keyType.equals("hashrange"))
-          mappingBuilder.setHashRangeKeySchema(tableName, keyName, keyAttrType);
-      }
-      log.debug("Table key schemas have been set.");
-    
-      // Retrieving attributes
-        fieldElements = tableElement.getChildren("attribute");
-        for(Element fieldElement : fieldElements) {
-          String attributeName  = fieldElement.getAttributeValue("name");
+      boolean keys = false;
+      for (Element tableElement : tableElements) {
+
+        String tableName = tableElement.getAttributeValue("name");
+        long readCapacUnits = Long.parseLong(tableElement
+            .getAttributeValue("readcunit"));
+        long writeCapacUnits = Long.parseLong(tableElement
+            .getAttributeValue("writecunit"));
+        this.packageName = tableElement.getAttributeValue("package");
+
+        mappingBuilder.setProvisionedThroughput(tableName, readCapacUnits,
+            writeCapacUnits);
+        log.debug("Table properties have been set for name, package and provisioned throughput.");
+
+        // Retrieving attributes
+        List<Element> fieldElements = tableElement.getChildren("attribute");
+        for (Element fieldElement : fieldElements) {
+          String key = fieldElement.getAttributeValue("key");
+          String attributeName = fieldElement.getAttributeValue("name");
           String attributeType = fieldElement.getAttributeValue("type");
-          mappingBuilder.addAttribute(tableName, attributeName, attributeType, 0);
+          mappingBuilder.addAttribute(tableName, attributeName, attributeType);
+          // Retrieving key's features
+          if (key != null) {
+            mappingBuilder.setKeySchema(tableName, attributeName, key);
+            keys = true;
+          }
         }
-        log.info("Table attributes have been read.");
+        log.debug("Attributes for table '{}' have been read.", tableName);
+        if (!keys)
+          log.warn("Keys for table '{}' have NOT been set.", tableName);
       }
-
     } catch(IOException ex) {
-      log.info("Error while performing xml mapping.");
-      ex.printStackTrace();
-      throw ex;
-
+      log.error("Error while performing xml mapping.", ex.getMessage());
+      throw new RuntimeException(ex);
     } catch(Exception ex) {
-      ex.printStackTrace();
+      log.error("An error occured whilst reading the xml mapping file!", ex.getMessage());
       throw new IOException(ex);
     }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBKey.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBKey.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBKey.java
index b29786e..0aa8333 100644
--- a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBKey.java
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/query/DynamoDBKey.java
@@ -18,18 +18,22 @@
 
 package org.apache.gora.dynamodb.query;
 
-public class DynamoDBKey<H, R>  {
-  
+/**
+ * Class abstracting a composed DynamoDB key.
+ * @param <H>
+ * @param <R>
+ */
+public class DynamoDBKey<H, R> {  
   /**
    * Hash key used for a specific table 
    */
   private H hashKey;
-  
+
   /**
    * Range key used for a specific table
    */
   private R rangeKey;
-  
+
   /**
    * Gets hash key
    * @return
@@ -37,7 +41,7 @@ public class DynamoDBKey<H, R>  {
   public H getHashKey() {
     return hashKey;
   }
-  
+
   /**
    * Sets hash key
    * @param hashKey
@@ -45,7 +49,7 @@ public class DynamoDBKey<H, R>  {
   public void setHashKey(H hashKey) {
     this.hashKey = hashKey;
   }
-  
+
   /**
    * Gets range key
    * @return
@@ -53,7 +57,7 @@ public class DynamoDBKey<H, R>  {
   public R getRangeKey() {
     return rangeKey;
   }
-  
+
   /**
    * Sets range key
    * @param rangeKey
@@ -61,4 +65,13 @@ public class DynamoDBKey<H, R>  {
   public void setRangeKey(R rangeKey) {
     this.rangeKey = rangeKey;
   }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append('[').append(hashKey != null? hashKey.toString():":");
+    sb.append(rangeKey != null? ":" + rangeKey.toString():"");
+    sb.append(']');
+    return sb.toString();
+  }
 }


[2/5] gora git commit: GORA-362 Refactor gora-dynamodb to support avro serialization

Posted by le...@apache.org.
http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java
index 83e904f..8fbf68c 100644
--- a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBStore.java
@@ -18,25 +18,29 @@
 
 package org.apache.gora.dynamodb.store;
 
-import java.io.FileNotFoundException;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.CLI_TYP_PROP;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.CONSISTENCY_READS;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.CONSISTENCY_READS_TRUE;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.ENDPOINT_PROP;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.MAPPING_FILE;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.PREF_SCH_NAME;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.SERIALIZATION_TYPE;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.SLEEP_DELETE_TIME;
+import static org.apache.gora.dynamodb.store.DynamoDBUtils.WAIT_TIME;
+
 import java.io.IOException;
-import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 import java.util.Properties;
-import java.lang.reflect.InvocationTargetException;
-import java.lang.reflect.Method;
 
-import org.apache.gora.dynamodb.query.DynamoDBKey;
-import org.apache.gora.dynamodb.query.DynamoDBQuery;
-import org.apache.gora.dynamodb.query.DynamoDBResult;
 import org.apache.gora.dynamodb.store.DynamoDBMapping.DynamoDBMappingBuilder;
 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.store.ws.impl.WSDataStoreBase;
+import org.apache.gora.store.DataStore;
 import org.apache.gora.util.GoraException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -45,781 +49,525 @@ import org.jdom.Element;
 import org.jdom.input.SAXBuilder;
 
 import com.amazonaws.AmazonServiceException;
-import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.services.dynamodb.AmazonDynamoDB;
-import com.amazonaws.services.dynamodb.AmazonDynamoDBAsyncClient;
-import com.amazonaws.services.dynamodb.AmazonDynamoDBClient;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBMapper;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBQueryExpression;
-import com.amazonaws.services.dynamodb.datamodeling.DynamoDBScanExpression;
-import com.amazonaws.services.dynamodb.model.CreateTableRequest;
-import com.amazonaws.services.dynamodb.model.DeleteTableRequest;
-import com.amazonaws.services.dynamodb.model.DeleteTableResult;
-import com.amazonaws.services.dynamodb.model.DescribeTableRequest;
-import com.amazonaws.services.dynamodb.model.KeySchema;
-import com.amazonaws.services.dynamodb.model.ProvisionedThroughput;
-import com.amazonaws.services.dynamodb.model.ResourceNotFoundException;
-import com.amazonaws.services.dynamodb.model.TableDescription;
-import com.amazonaws.services.dynamodb.model.TableStatus;
-
-
-public class DynamoDBStore<K, T extends Persistent> extends WSDataStoreBase<K, T> {
-  
-  /**
-   * Helper to write useful information into the logs
-   */
-  public static final Logger LOG = LoggerFactory.getLogger(DynamoDBStore.class);
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.model.DeleteTableRequest;
+import com.amazonaws.services.dynamodbv2.model.DeleteTableResult;
+import com.amazonaws.services.dynamodbv2.model.DescribeTableRequest;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
 
-  /**
-   * Schema name which will be used from within the data store.
-   * If not set, all the available schemas from the mapping file will be used.
-   */
-  private static String preferredSchema;
-  
-  /**
-   * The mapping file to create the tables from
-   */
-  private static final String MAPPING_FILE = "gora-dynamodb-mapping.xml";
+/**
+ * Class for using DynamoDBStores
+ * 
+ * @param <K>
+ * @param <T>
+ */
+public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
 
-  /**
-   * Default times to wait while requests are performed
-   */
-  private static long waitTime = 10L * 60L * 1000L;
-  private static long sleepTime = 1000L * 20L;
-  private static long sleepDeleteTime = 1000L * 10L;
+  /** Handler for different serialization modes. */
+  private IDynamoDB<K, T> dynamoDbStore;
 
-  /**
-   * AWS Credential file name.
-   */
-  private static String awsCredentialsProperties = "AwsCredentials.properties";
-  
-  /**
-   * Name of the cloud database provider.
-   */
-  private static String wsProvider = "Amazon.Web.Services";
-  
-  /**
-   * Parameter to decide what type of Amazon DynamoDB client to use
-   */
-  private static String CLI_TYP_PROP = "gora.dynamodb.client";
-  
-  /**
-   * Parameter to decide where the data store will make its computations
-   */
-  private static String ENDPOINT_PROP = "gora.dynamodb.endpoint";
-  
-  /**
-   * Parameter to decide which schema will be used
-   */
-  private static String PREF_SCH_NAME = "preferred.schema.name";
-  
-  /**
-   * Parameter to decide how reads will be made i.e. using strong consistency or eventual consistency. 
-   */
-  private static String CONSISTENCY_READS = "gora.dynamodb.consistent.reads";
+  /** Helper to write useful information into the logs. */
+  public static final Logger LOG = LoggerFactory.getLogger(DynamoDBStore.class);
 
   /**
    * The mapping object that contains the mapping file
    */
   private DynamoDBMapping mapping;
-  
+
   /**
-   * Amazon DynamoDB client which can be asynchronous or nor   
+   * Amazon DynamoDB client which can be asynchronous or not
    */
   private AmazonDynamoDB dynamoDBClient;
- 
+
   /**
    * Contains the consistency level to be used
    */
   private String consistency;
-  
-  /**
-   * TODO This would be useful for the batch read/write operations
-   * Contains the elements to be written or read from the data store
-   */
-  //private Map<K, T> buffer = new LinkedHashMap<K, T>();
-  
-  /**
-   * The class that will be persisted
-   */
-  Class<T> persistentClass;  
 
-  /**
-   * Constructor
-   */
-  public DynamoDBStore(){
-  }
+  /** Specifies how the objects will be serialized inside DynamoDb. */
+  private DynamoDBUtils.DynamoDBType serializationType;
 
   /**
-   * Initialize the data store by reading the credentials, setting the cloud provider,
-   * setting the client's properties up, setting the end point and reading the mapping file  
+   * Schema name which will be used from within the data store. If not set, all
+   * the available schemas from the mapping file will be used.
    */
-  public void initialize(Class<K> keyClass, Class<T> pPersistentClass,
-       Properties properties) {
-    try {
-      LOG.debug("Initializing DynamoDB store");
-      getCredentials();
-      setWsProvider(wsProvider);
-      preferredSchema = DataStoreFactory.findProperty(properties, this, PREF_SCH_NAME, null);
-      dynamoDBClient = getClient(DataStoreFactory.findProperty(properties, this, CLI_TYP_PROP, null),(AWSCredentials)getConf());
-      dynamoDBClient.setEndpoint(DataStoreFactory.findProperty(properties, this, ENDPOINT_PROP, null));
-      mapping = readMapping();
-      consistency = DataStoreFactory.findProperty(properties, this, CONSISTENCY_READS, null);
-      persistentClass = pPersistentClass;
-    }
-    catch (Exception e) {
-      LOG.error("Error while initializing DynamoDB store");
-      LOG.error(e.getMessage(), e);
-    }
-  }
-  
-   /**
-    * Method to create the specific client to be used
-    * @param clientType
-    * @param credentials
-    * @return
-    */
-  public AmazonDynamoDB getClient(String clientType, AWSCredentials credentials){
-    if (clientType.equals("sync"))
-      return new AmazonDynamoDBClient(credentials);
-    if (clientType.equals("async"))
-      return new AmazonDynamoDBAsyncClient(credentials);
-    return null;
+  private String preferredSchema;
+
+  @Override
+  public void close() {
+    dynamoDbStore.close();
   }
-  
+
   /**
-   * Reads the schema file and converts it into a data structure to be used
-   * @param pMapFile	The schema file to be mapped into a table
-   * @return DynamoDBMapping	Object containing all necessary information to create tables
+   * Creates the table within the data store for a preferred schema or for a
+   * group of schemas defined within the mapping file
    * @throws IOException
    */
-  @SuppressWarnings("unchecked")
-  private DynamoDBMapping readMapping() throws IOException {
+  @Override
+  public void createSchema() {
+    dynamoDbStore.createSchema();
+  }
 
-    DynamoDBMappingBuilder mappingBuilder = new DynamoDBMappingBuilder();
+  @Override
+  public boolean delete(K key) {
+    return dynamoDbStore.delete(key);
+  }
 
-    try {
-      SAXBuilder builder = new SAXBuilder();
-      Document doc = builder.build(getClass().getClassLoader().getResourceAsStream(MAPPING_FILE));
-      
-      Element root = doc.getRootElement();
+  @Override
+  public long deleteByQuery(Query<K, T> query) {
+    return dynamoDbStore.deleteByQuery(query);
+  }
 
-      List<Element> tableElements = root.getChildren("table");
-      for(Element tableElement : tableElements) {
-    
-        String tableName = tableElement.getAttributeValue("name");
-        long readCapacUnits = Long.parseLong(tableElement.getAttributeValue("readcunit"));
-        long writeCapacUnits = Long.parseLong(tableElement.getAttributeValue("writecunit"));
-    
-        mappingBuilder.setTableName(tableName);
-        mappingBuilder.setProvisionedThroughput(tableName, readCapacUnits, writeCapacUnits);
-        LOG.debug("Basic table properties have been set: Name, and Provisioned throughput.");
-    
-        // Retrieving key's features
-        List<Element> fieldElements = tableElement.getChildren("key");
-        for(Element fieldElement : fieldElements) {
-          String keyName  = fieldElement.getAttributeValue("name");
-          String keyType  = fieldElement.getAttributeValue("type");
-          String keyAttrType  = fieldElement.getAttributeValue("att-type");
-          if(keyType.equals("hash"))
-            mappingBuilder.setHashKeySchema(tableName, keyName, keyAttrType);
-          else if(keyType.equals("hashrange"))
-            mappingBuilder.setHashRangeKeySchema(tableName, keyName, keyAttrType);
-        }
-        LOG.debug("Table key schemas have been set.");
-    
-        // Retrieving attributes
-        fieldElements = tableElement.getChildren("attribute");
-        for(Element fieldElement : fieldElements) {
-          String attributeName  = fieldElement.getAttributeValue("name");
-          String attributeType = fieldElement.getAttributeValue("type");
-          mappingBuilder.addAttribute(tableName, attributeName, attributeType, 0);
-        }
-        LOG.debug("Table attributes have been read.");
-      }
+  @Override
+  public void deleteSchema() {
+    if (getDynamoDbMapping().getTables().isEmpty())
+      throw new IllegalStateException("There are not tables defined.");
+    if (preferredSchema == null) {
+      LOG.debug("Delete schemas");
+      if (getDynamoDbMapping().getTables().isEmpty())
+        throw new IllegalStateException("There are not tables defined.");
+      // read the mapping object
+      for (String tableName : getDynamoDbMapping().getTables().keySet())
+        executeDeleteTableRequest(tableName);
+      LOG.debug("All schemas deleted successfully.");
+    } else {
+      LOG.debug("create schema " + preferredSchema);
+      executeDeleteTableRequest(preferredSchema);
+    }
+  }
 
-    } catch(IOException ex) {
-      LOG.error("Error while performing xml mapping.");
-      ex.printStackTrace();
-      throw ex;
+  @Override
+  public Result<K, T> execute(Query<K, T> query) {
+    return dynamoDbStore.execute(query);
+  }
 
-    } catch(Exception ex) {
-      LOG.error("Error while performing xml mapping.");
-      ex.printStackTrace();
-      throw new IOException(ex);
-    }
+  @Override
+  public void flush() {
+    dynamoDbStore.flush();
+  }
 
-    return mappingBuilder.build();
+  @Override
+  public T get(K key) {
+    return dynamoDbStore.get(key);
   }
-  
-  /**
-   * Creates the AWSCredentials object based on the properties file.
-   * @return AWSCredentials object
-   * @throws FileNotFoundException
-   * @throws IllegalArgumentException
-   * @throws IOException
-   */
-  private AWSCredentials getCredentials() throws FileNotFoundException, 
-    IllegalArgumentException, IOException {
-    
-  if(authentication == null){
-    InputStream awsCredInpStr = getClass().getClassLoader().getResourceAsStream(awsCredentialsProperties);
-    if (awsCredInpStr == null)
-      LOG.info("AWS Credentials File was not found on the classpath!");
-      AWSCredentials credentials = new PropertiesCredentials(awsCredInpStr);
-      setConf(credentials);
+
+  @Override
+  public T get(K key, String[] fields) {
+    return dynamoDbStore.get(key, fields);
   }
-  return (AWSCredentials)authentication;
+
+  @Override
+  public BeanFactory<K, T> getBeanFactory() {
+    // TODO Auto-generated method stub
+    return null;
   }
 
-  /**
-   * Builds a DynamoDB query from a generic Query object
-   * @param query	Generic query object
-   * @return	DynamoDBQuery 
-   */
-  private DynamoDBQuery<K, T> buildDynamoDBQuery(Query<K, T> query){
-    if(getSchemaName() == null) throw new IllegalStateException("There is not a preferred schema defined.");
-    
-      DynamoDBQuery<K, T> dynamoDBQuery = new DynamoDBQuery<K, T>();
-      dynamoDBQuery.setKeySchema(mapping.getKeySchema(getSchemaName()));
-      dynamoDBQuery.setQuery(query);
-      dynamoDBQuery.setConsistencyReadLevel(getConsistencyReads());
-      dynamoDBQuery.buildExpression();
-    
-      return dynamoDBQuery;
-  }
-  
-  /**
-   * Gets consistency level for reads
-   * @return True for strong consistency or false for eventual consistent reads
-   */
-  private boolean getConsistencyReads(){
-    if(consistency != null)
-      if(consistency.equals("true")) 
-        return true;
-    return false;
+  @Override
+  public Class<K> getKeyClass() {
+    // TODO Auto-generated method stub
+    return null;
   }
-  
-  /**
-   * Executes a query after building a DynamoDB specific query based on the received one
-   */
+
   @Override
-  public Result<K, T> execute(Query<K, T> query) {
-    DynamoDBQuery<K, T> dynamoDBQuery = buildDynamoDBQuery(query);
-    DynamoDBMapper mapper = new DynamoDBMapper(dynamoDBClient);
-    List<T> objList = null;
-    if (DynamoDBQuery.getType().equals(DynamoDBQuery.RANGE_QUERY))
-      objList = mapper.query(persistentClass, (DynamoDBQueryExpression)dynamoDBQuery.getQueryExpression());
-      if (DynamoDBQuery.getType().equals(DynamoDBQuery.SCAN_QUERY))
-        objList = mapper.scan(persistentClass, (DynamoDBScanExpression)dynamoDBQuery.getQueryExpression());
-        return new DynamoDBResult<K, T>(this, query, objList);  
-  }
-  
+  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> arg0)
+      throws IOException {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
   @Override
-  public T get(K key, String[] fields) {
-   /* 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 Class<T> getPersistentClass() {
+    // TODO Auto-generated method stub
     return null;
   }
 
   @Override
-  /**
-   * Gets the object with the specific key
-   * @throws IOException
-   */
-  public T get(K key) {
-    T object = null;
+  public String getSchemaName() {
+    return this.getPreferredSchema();
+  }
+
+  @Override
+  public void initialize(Class<K> keyClass, Class<T> persistentClass,
+      Properties properties) {
     try {
-      Object rangeKey;
-      rangeKey = getRangeKey(key);
-      Object hashKey = getHashKey(key);
-      if (hashKey != null){
-        DynamoDBMapper mapper = new DynamoDBMapper(dynamoDBClient);
-       if (rangeKey != null)
-        object = mapper.load(persistentClass, hashKey, rangeKey);
-      else
-        object = mapper.load(persistentClass, hashKey);
-      }
-      else
-        throw new GoraException("Error while retrieving keys from object: " + key.toString());
-    } catch (IllegalArgumentException e) {
-      e.printStackTrace();
-    } catch (IllegalAccessException e) {
-      e.printStackTrace();
-    } catch (InvocationTargetException e) {
-      e.printStackTrace();
-    } catch (GoraException ge){
-      LOG.error(ge.getMessage(), ge);
+      LOG.debug("Initializing DynamoDB store");
+      setDynamoDBProperties(properties);
+
+      dynamoDbStore = DynamoDBFactory.buildDynamoDBStore(getSerializationType());
+      dynamoDbStore.setDynamoDBStoreHandler(this);
+      dynamoDbStore.initialize(keyClass, persistentClass, properties);
+    } catch (Exception e) {
+      LOG.error("Error while initializing DynamoDB store", e.getMessage());
+      throw new RuntimeException(e);
     }
-    return object;
   }
-    
-  /**
-   * Creates a new DynamoDBQuery
-   */
-  public Query<K, T> newQuery() {
-    Query<K,T> query = new DynamoDBQuery<K, T>(this);
-    //query.setFields(getFieldsToQuery(null));
-    return query;
+
+  private void setDynamoDBProperties(Properties properties) throws IOException {
+    setSerializationType(properties.getProperty(SERIALIZATION_TYPE));
+    PropertiesCredentials creds = DynamoDBUtils.getCredentials(this.getClass());
+    setPreferredSchema(properties.getProperty(PREF_SCH_NAME));
+    setDynamoDBClient(DynamoDBUtils.getClient(
+        properties.getProperty(CLI_TYP_PROP), creds));
+    getDynamoDBClient().setEndpoint(properties.getProperty(ENDPOINT_PROP));
+    setDynamoDbMapping(readMapping());
+    setConsistency(properties.getProperty(CONSISTENCY_READS));
   }
 
-  /**
-   * Gets the preferred schema
-   */
-  public String getSchemaName() {
-    if (preferredSchema != null)
-      return preferredSchema;
-    return null;
+  @Override
+  public K newKey() {
+    return dynamoDbStore.newKey();
   }
-  
-  /**
-   * Sets the preferred schema
-   * @param pSchemaName
-   */
-  public void setSchemaName(String pSchemaName){
-    preferredSchema = pSchemaName;
+
+  @Override
+  public T newPersistent() {
+    return dynamoDbStore.newPersistent();
+  }
+
+  @Override
+  public Query<K, T> newQuery() {
+    return dynamoDbStore.newQuery();
+  }
+
+  @Override
+  public void put(K key, T value) {
+    dynamoDbStore.put(key, value);
   }
-  
+
+
+
   /**
-   * Creates the table within the data store for a preferred schema or 
-   * for a group of schemas defined withing the mapping file
+   * Verifies if the specified schemas exist
+   * 
    * @throws IOException
    */
   @Override
-  public void createSchema() {
-    LOG.info("Creating schema");
-    if (mapping.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
-    if (preferredSchema == null){
-      LOG.debug("create schemas");
+  public boolean schemaExists() {
+    LOG.info("Verifying schemas.");
+    TableDescription success = null;
+    if (getDynamoDbMapping().getTables().isEmpty())
+      throw new IllegalStateException("There are not tables defined.");
+    if (getPreferredSchema() == null) {
+      LOG.debug("Verifying schemas");
+      if (getDynamoDbMapping().getTables().isEmpty())
+        throw new IllegalStateException("There are not tables defined.");
       // read the mapping object
-      for(String tableName : mapping.getTables().keySet())
-        executeCreateTableRequest(tableName);
-        LOG.debug("tables created successfully.");
-    }
-    else{
-      LOG.debug("create schema " + preferredSchema);
-      executeCreateTableRequest(preferredSchema);
+      for (String tableName : getDynamoDbMapping().getTables().keySet()) {
+        success = getTableSchema(tableName);
+        if (success == null)
+          return false;
+      }
+    } else {
+      LOG.info("Verifying schema " + preferredSchema);
+      success = getTableSchema(preferredSchema);
     }
+    LOG.info("Finished verifying schemas.");
+    return (success != null) ? true : false;
   }
-  
-  /**
-   * Executes a create table request using the DynamoDB client and waits
-   * the default time until it's been created.
-   * @param tableName
-   */
-  private void executeCreateTableRequest(String tableName){
-    CreateTableRequest createTableRequest = getCreateTableRequest(tableName,
-      mapping.getKeySchema(tableName), mapping.getProvisionedThroughput(tableName));
-    // use the client to perform the request
-    dynamoDBClient.createTable(createTableRequest).getTableDescription();
-    // wait for table to become active
-    waitForTableToBecomeAvailable(tableName);
-    LOG.info(tableName + "Schema now available");
-  }
-  
-  /**
-   * Builds the necessary requests to create tables 
-   * @param tableName
-   * @param keySchema
-   * @param proThrou
-   * @return
-   */
-  private CreateTableRequest getCreateTableRequest(String tableName, KeySchema keySchema, ProvisionedThroughput proThrou){
-    CreateTableRequest createTableRequest = new CreateTableRequest();
-    createTableRequest.setTableName(tableName);
-    createTableRequest.setKeySchema(keySchema);
-    createTableRequest.setProvisionedThroughput(proThrou);
-    return createTableRequest;
-  }
-  
-  /**
-   * Deletes all tables present in the mapping object.
+
+  @Override
+  public void setBeanFactory(BeanFactory<K, T> arg0) {
+    // TODO Auto-generated method stub
+  }
+
+  @Override
+  public void setKeyClass(Class<K> arg0) {
+    dynamoDbStore.setKeyClass(arg0);
+  }
+
+  @Override
+  public void setPersistentClass(Class<T> arg0) {
+    dynamoDbStore.setPersistentClass(arg0);
+  }
+
+  @Override
+  public void truncateSchema() {
+    // TODO Auto-generated method stub
+  }
+
+  /** 
+   * Reads the schema file and converts it into a data structure to be used
+   * 
+   * @param pMapFile
+   *          The schema file to be mapped into a table
+   * @return DynamoDBMapping Object containing all necessary information to
+   *         create tables
    * @throws IOException
    */
-  @Override
-  public void deleteSchema() {
-    if (mapping.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
-    if (preferredSchema == null){
-      LOG.debug("Delete schemas");
-      if (mapping.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
-      // read the mapping object
-      for(String tableName : mapping.getTables().keySet())
-        executeDeleteTableRequest(tableName);
-        LOG.debug("All schemas deleted successfully.");
-    }
-      else{
-        LOG.debug("create schema " + preferredSchema);
-        executeDeleteTableRequest(preferredSchema);
+  @SuppressWarnings("unchecked")
+  private DynamoDBMapping readMapping() throws IOException {
+
+    DynamoDBMappingBuilder mappingBuilder = new DynamoDBMappingBuilder();
+
+    try {
+      SAXBuilder builder = new SAXBuilder();
+      Document doc = builder.build(getClass().getClassLoader()
+          .getResourceAsStream(MAPPING_FILE));
+      if (doc == null || doc.getRootElement() == null)
+        throw new GoraException("Unable to load " + MAPPING_FILE
+            + ". Please check its existance!");
+
+      Element root = doc.getRootElement();
+      List<Element> tableElements = root.getChildren("table");
+      boolean keys = false;
+      for (Element tableElement : tableElements) {
+
+        String tableName = tableElement.getAttributeValue("name");
+        long readCapacUnits = Long.parseLong(tableElement
+            .getAttributeValue("readcunit"));
+        long writeCapacUnits = Long.parseLong(tableElement
+            .getAttributeValue("writecunit"));
+
+        mappingBuilder.setProvisionedThroughput(tableName, readCapacUnits,
+            writeCapacUnits);
+        LOG.debug("Basic table properties have been set: Name, and Provisioned throughput.");
+
+        // Retrieving attributes
+        List<Element> fieldElements = tableElement.getChildren("attribute");
+        for (Element fieldElement : fieldElements) {
+          String key = fieldElement.getAttributeValue("key");
+          String attributeName = fieldElement.getAttributeValue("name");
+          String attributeType = fieldElement.getAttributeValue("type");
+          mappingBuilder.addAttribute(tableName, attributeName, attributeType);
+          // Retrieving key's features
+          if (key != null) {
+            mappingBuilder.setKeySchema(tableName, attributeName, key);
+            keys = true;
+          }
+        }
+        LOG.debug("Attributes for table '" + tableName + "' have been read.");
+        if (!keys)
+          LOG.warn("Keys for table '" + tableName + "' have NOT been set.");
+      }
+    } catch (IOException ex) {
+      LOG.error("Error while performing xml mapping.", ex.getMessage());
+      throw new IOException(ex);
+    } catch (Exception ex) {
+      LOG.error("Error while performing xml mapping.", ex.getMessage());
+      throw new RuntimeException(ex);
     }
+
+    return mappingBuilder.build();
   }
-  
+
   /**
    * Executes a delete table request using the DynamoDB client
+   * 
    * @param tableName
    */
-  public void executeDeleteTableRequest(String pTableName){
-    try{
-      DeleteTableRequest deleteTableRequest = new DeleteTableRequest().withTableName(pTableName);
-      DeleteTableResult result = dynamoDBClient.deleteTable(deleteTableRequest);
+  public void executeDeleteTableRequest(String pTableName) {
+    try {
+      DeleteTableRequest deleteTableRequest = new DeleteTableRequest()
+          .withTableName(pTableName);
+      DeleteTableResult result = getDynamoDBClient().deleteTable(
+          deleteTableRequest);
       waitForTableToBeDeleted(pTableName);
-      LOG.debug("Schema: " + result.getTableDescription() + " deleted successfully.");
-    }
-    catch(Exception e){
-      LOG.debug("Schema: " + pTableName + " deleted.");
-      e.printStackTrace();
+      LOG.debug("Schema: " + result.getTableDescription()
+      + " deleted successfully.");
+    } catch (Exception e) {
+      LOG.debug("Schema: {} deleted.", pTableName, e.getMessage());
+      throw new RuntimeException(e);
     }
   }
 
+
+
   /**
    * Waits up to 6 minutes to confirm if a table has been deleted or not
+   * 
    * @param pTableName
    */
-  private void waitForTableToBeDeleted(String pTableName){
+  private void waitForTableToBeDeleted(String pTableName) {
     LOG.debug("Waiting for " + pTableName + " to be deleted.");
     long startTime = System.currentTimeMillis();
-    long endTime = startTime + waitTime;
+    long endTime = startTime + WAIT_TIME;
     while (System.currentTimeMillis() < endTime) {
-      try {Thread.sleep(sleepDeleteTime);} catch (Exception e) {}
       try {
-        DescribeTableRequest request = new DescribeTableRequest().withTableName(pTableName);
-        TableDescription tableDescription = dynamoDBClient.describeTable(request).getTable();
+        Thread.sleep(SLEEP_DELETE_TIME);
+      } catch (Exception e) {
+      }
+      try {
+        DescribeTableRequest request = new DescribeTableRequest()
+            .withTableName(pTableName);
+        TableDescription tableDescription = getDynamoDBClient().describeTable(
+            request).getTable();
         String tableStatus = tableDescription.getTableStatus();
         LOG.debug(pTableName + " - current state: " + tableStatus);
       } catch (AmazonServiceException ase) {
         if (ase.getErrorCode().equalsIgnoreCase("ResourceNotFoundException") == true)
           return;
-        ase.printStackTrace();
+        LOG.error(ase.getMessage());
       }
     }
     LOG.debug(pTableName + " deleted.");
   }
-  
-  /**
-   * Waits up to 6 minutes to confirm if a table has been created or not
-   * @param pTableName
-   */
-  private void waitForTableToBecomeAvailable(String tableName) {
-    LOG.debug("Waiting for " + tableName + " to become available");
-    long startTime = System.currentTimeMillis();
-    long endTime = startTime + waitTime;
-    while (System.currentTimeMillis() < endTime) {
-      try {Thread.sleep(sleepTime);} catch (Exception e) {}
-      try {
-        DescribeTableRequest request = new DescribeTableRequest().withTableName(tableName);
-        TableDescription tableDescription = dynamoDBClient.describeTable(request).getTable();
-        String tableStatus = tableDescription.getTableStatus();
-        LOG.debug(tableName + " - current state: " + tableStatus);
-        if (tableStatus.equals(TableStatus.ACTIVE.toString())) return;
-      } catch (AmazonServiceException ase) {
-        if (ase.getErrorCode().equalsIgnoreCase("ResourceNotFoundException") == false) throw ase;
-      }
-    }
-    throw new RuntimeException("Table " + tableName + " never became active");
-  }
-
-  /**
-   * Verifies if the specified schemas exist
-   * @throws IOException
-   */
-  @Override
-  public boolean schemaExists() {
-    LOG.info("Verifying schemas.");
-  TableDescription success = null;
-  if (mapping.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
-  if (preferredSchema == null){
-    LOG.debug("Verifying schemas");
-    if (mapping.getTables().isEmpty())  throw new IllegalStateException("There are not tables defined.");
-    // read the mapping object
-    for(String tableName : mapping.getTables().keySet()){
-        success = getTableSchema(tableName);
-        if (success == null) return false;
-      }
-    }
-    else{
-      LOG.info("Verifying schema " + preferredSchema);
-      success = getTableSchema(preferredSchema);
-  }
-    LOG.info("Finished verifying schemas.");
-    return (success != null)? true: false;
-  }
 
   /**
    * Retrieves the table description for the specific resource name
+   * 
    * @param tableName
    * @return
    */
-  private TableDescription getTableSchema(String tableName){
+  private TableDescription getTableSchema(String tableName) {
     TableDescription tableDescription = null;
-    try{
-      DescribeTableRequest describeTableRequest = new DescribeTableRequest().withTableName(tableName);
-      tableDescription = dynamoDBClient.describeTable(describeTableRequest).getTable();
-    }
-    catch(ResourceNotFoundException e){
+    try {
+      DescribeTableRequest describeTableRequest = new DescribeTableRequest()
+          .withTableName(tableName);
+      tableDescription = getDynamoDBClient()
+          .describeTable(describeTableRequest).getTable();
+    } catch (ResourceNotFoundException e) {
       LOG.error("Error while getting table schema: " + tableName);
       return tableDescription;
     }
     return tableDescription;
   }
+
   /**
-   * Returns a new instance of the key object.
-   * @throws IOException
+   * Gets a specific table key schema.
+   * 
+   * @param tableName
+   *          from which key schema is to be obtained.
+   * @return KeySchema from table.
    */
-  @Override
-  public K newKey() {
-    // TODO Auto-generated method stub
-    return null;
+  public ArrayList<KeySchemaElement> getTableKeySchema(String tableName) {
+    return getDynamoDbMapping().getKeySchema(tableName);
   }
 
   /**
-   * Returns a new persistent object
-   * @throws IOException
+   * Gets the provisioned throughput for a specific table.
+   * 
+   * @param tableName
+   *          to get the ProvisionedThroughput.
+   * @return ProvisionedThroughput for a specific table
    */
-  @Override
-  public T newPersistent() {
-    T obj = null;
-    try {
-      obj = persistentClass.newInstance();
-    } catch (InstantiationException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    } catch (IllegalAccessException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-    return obj;
+  public ProvisionedThroughput getTableProvisionedThroughput(String tableName) {
+    return getDynamoDbMapping().getProvisionedThroughput(tableName);
+  }
+  /**
+   * Returns a table attribues.
+   * @param tableName
+   * @return
+   */
+  public Map<String, String> getTableAttributes(String tableName) {
+    return getDynamoDbMapping().getItems(tableName);
   }
 
   /**
-   * Puts an object identified by a key
-   * @throws IOException
+   * Gets consistency level for reads
+   * 
+   * @return True for strong consistency or false for eventual consistent reads
    */
-  @Override
-  public void put(K key, T obj) {
-    try{
-      Object rangeKey = getRangeKey(key);
-      Object hashKey = getHashKey(key);
-      // if the key does not have these attributes then try to get them from the object
-      if (hashKey == null)
-        hashKey = getHashKey(obj);
-      if (rangeKey == null)
-        rangeKey = getRangeKey(obj);
-      if (hashKey != null){
-        DynamoDBMapper mapper = new DynamoDBMapper(dynamoDBClient);
-        if (rangeKey != null)
-          mapper.load(persistentClass, hashKey.toString(), rangeKey.toString());
-        else
-          mapper.load(persistentClass, hashKey.toString());
-          mapper.save(obj);
-      }
-      else
-        throw new GoraException("Error while retrieving keys from object: " + obj.toString());
-    }catch(NullPointerException npe){
-      LOG.error("Error while putting an item. " + npe.toString());
-      npe.printStackTrace();
-    }catch(Exception e){
-      LOG.error("Error while putting an item. " + obj.toString());
-      e.printStackTrace();
-    }
+  public boolean getConsistencyReads() {
+    if (getConsistency() != null)
+      if (getConsistency().equals(CONSISTENCY_READS_TRUE))
+        return true;
+    return false;
   }
 
+
   /**
-   * Deletes the object using key
-   * @return true for a successful process  
-   * @throws IOException
+   * Set DynamoDBStore to be used.
+   * 
+   * @param iDynamoDB
    */
-  @Override
-  public boolean delete(K key) {
-    try{
-      T object = null;
-      Object rangeKey = null, hashKey = null;
-      DynamoDBMapper mapper = new DynamoDBMapper(dynamoDBClient);
-      for (Method met :key.getClass().getDeclaredMethods()){
-        if(met.getName().equals("getRangeKey")){
-          Object [] params = null;
-          rangeKey = met.invoke(key, params);
-          break;
-        }
-      }
-      for (Method met :key.getClass().getDeclaredMethods()){
-        if(met.getName().equals("getHashKey")){
-          Object [] params = null;
-          hashKey = met.invoke(key, params);
-          break;
-        }
-      }
-      if (hashKey == null) object = (T) mapper.load(persistentClass, key);
-      if (rangeKey == null)
-        object = (T) mapper.load(persistentClass, hashKey);
-      else
-        object = (T) mapper.load(persistentClass, hashKey, rangeKey);
-
-      if (object == null) return false;
-
-      // setting key for dynamodbMapper
-      mapper.delete(object);
-      return true;
-    }catch(Exception e){
-      LOG.error("Error while deleting value with key " + key.toString());
-      LOG.error(e.getMessage());
-      return false;
-    }
+  public void setDynamoDbStore(IDynamoDB<K, T> iDynamoDB) {
+    this.dynamoDbStore = iDynamoDB;
   }
-  
+
   /**
-   * Deletes items using a specific query
-   * @throws IOException
+   * @param serializationType
+   *          the serializationType to set
    */
-  @Override
-  @SuppressWarnings("unchecked")
-  public long deleteByQuery(Query<K, T> query) {
-    // TODO verify whether or not we are deleting a whole row
-    //String[] fields = getFieldsToQuery(query.getFields());
-    //find whether all fields are queried, which means that complete
-    //rows will be deleted
-    //boolean isAllFields = Arrays.equals(fields
-    //    , getBeanFactory().getCachedPersistent().getFields());
-    Result<K, T> result = execute(query);
-    ArrayList<T> deletes = new ArrayList<T>();
-    try {
-      while(result.next()) {
-        T resultObj = result.get(); 
-        deletes.add(resultObj);
-        
-        @SuppressWarnings("rawtypes")
-        DynamoDBKey dKey = new DynamoDBKey();
-        
-          dKey.setHashKey(getHashKey(resultObj));
-        
-        dKey.setRangeKey(getRangeKey(resultObj));
-        delete((K)dKey);
-      }
-    } catch (IllegalArgumentException e) {
-      e.printStackTrace();
-    } catch (IllegalAccessException e) {
-      e.printStackTrace();
-    } catch (InvocationTargetException e) {
-      e.printStackTrace();
-    } catch (Exception e) {
-      e.printStackTrace();
+  private void setSerializationType(String serializationType) {
+    if (serializationType == null || serializationType.isEmpty()
+        || serializationType.equals(DynamoDBUtils.AVRO_SERIALIZATION)) {
+      LOG.warn("Using AVRO serialization.");
+      this.serializationType = DynamoDBUtils.DynamoDBType.AVRO;
+    } else {
+      LOG.warn("Using DynamoDB serialization.");
+      this.serializationType = DynamoDBUtils.DynamoDBType.DYNAMO;
     }
-    return deletes.size();
   }
-  
+
   /**
-   * Gets a hash key from an object of type T
-   * @param obj	Object from which we will get a hash key
+   * Gets serialization type used inside DynamoDB module.
+   * 
    * @return
-   * @throws IllegalArgumentException
-   * @throws IllegalAccessException
-   * @throws InvocationTargetException
    */
-  private Object getHashKey(T obj) throws IllegalArgumentException, IllegalAccessException, InvocationTargetException{
-    Object hashKey = null;
-    for (Method met : obj.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getHashKey")){
-        Object [] params = null;
-        hashKey = met.invoke(obj, params);
-        break;
-      }
-    }
-    return hashKey;
+  private DynamoDBUtils.DynamoDBType getSerializationType() {
+    return serializationType;
   }
-  
+
   /**
-   * Gets a hash key from a key of type K
-   * @param obj	Object from which we will get a hash key
-   * @return
-   * @throws IllegalArgumentException
-   * @throws IllegalAccessException
-   * @throws InvocationTargetException
+   * @return the preferredSchema
    */
-  private Object getHashKey(K obj) throws IllegalArgumentException, IllegalAccessException, InvocationTargetException{
-    Object hashKey = null;
-    for (Method met : obj.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getHashKey")){
-        Object [] params = null;
-        hashKey = met.invoke(obj, params);
-        break;
-      }
-    }
-    return hashKey;
+  public String getPreferredSchema() {
+    return preferredSchema;
   }
-  
+
   /**
-   * Gets a range key from an object T
-   * @param obj	Object from which a range key will be extracted
-   * @return
-   * @throws IllegalArgumentException
-   * @throws IllegalAccessException
-   * @throws InvocationTargetException
+   * @param preferredSchema
+   *          the preferredSchema to set
    */
-  private Object getRangeKey(T obj) throws IllegalArgumentException, IllegalAccessException, InvocationTargetException{
-    Object rangeKey = null;
-    for (Method met : obj.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getRangeKey")){
-        Object [] params = null;
-        rangeKey = met.invoke(obj, params);
-        break;
-      }
-    }
-    return rangeKey;
+  public void setPreferredSchema(String preferredSchema) {
+    this.preferredSchema = preferredSchema;
   }
-  
+
   /**
-   * Gets a range key from a key obj
-   * @param obj	Object from which a range key will be extracted
+   * Gets DynamoDBClient.
+   * 
    * @return
-   * @throws IllegalArgumentException
-   * @throws IllegalAccessException
-   * @throws InvocationTargetException
    */
-  private Object getRangeKey(K obj) throws IllegalArgumentException, IllegalAccessException, InvocationTargetException{
-    Object rangeKey = null;
-    for (Method met : obj.getClass().getDeclaredMethods()){
-      if(met.getName().equals("getRangeKey")){
-        Object [] params = null;
-        rangeKey = met.invoke(obj, params);
-        break;
-      }
-    }
-    return rangeKey;
+  public AmazonDynamoDB getDynamoDbClient() {
+    return getDynamoDBClient();
   }
-  
-  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query) throws IOException {
-    // TODO Auto-generated method stub
-    return null;
+
+
+  /**
+   * @return the mapping
+   */
+  public DynamoDBMapping getDynamoDbMapping() {
+    return mapping;
   }
-  @Override
+
   /**
-   * flushes objects to DynamoDB
-   * @throws IOException
+   * @param mapping
+   *          the mapping to set
    */
-  public void flush() {
-    // TODO Auto-generated method stub
+  public void setDynamoDbMapping(DynamoDBMapping mapping) {
+    this.mapping = mapping;
   }
 
-  public void setBeanFactory(BeanFactory<K, T> beanFactory) {
-    // TODO Auto-generated method stub
+  /**
+   * @return the consistency
+   */
+  public String getConsistency() {
+    return consistency;
   }
 
-  public BeanFactory<K, T> getBeanFactory() {
-    // TODO Auto-generated method stub
-    return null;
+  /**
+   * @param consistency
+   *          the consistency to set
+   */
+  public void setConsistency(String consistency) {
+    this.consistency = consistency;
   }
 
-  @Override
   /**
-   * Closes the data store.
+   * @return the dynamoDBClient
    */
-  public void close() {
-    LOG.debug("Datastore closed.");
-    flush();
+  public AmazonDynamoDB getDynamoDBClient() {
+    return dynamoDBClient;
+  }
+
+  /**
+   * @param dynamoDBClient
+   *          the dynamoDBClient to set
+   */
+  public void setDynamoDBClient(AmazonDynamoDB dynamoDBClient) {
+    this.dynamoDBClient = dynamoDBClient;
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBUtils.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBUtils.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBUtils.java
new file mode 100644
index 0000000..4fdf5f9
--- /dev/null
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/DynamoDBUtils.java
@@ -0,0 +1,229 @@
+/*
+ * 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.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Map;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.amazonaws.AmazonServiceException;
+import com.amazonaws.auth.AWSCredentials;
+import com.amazonaws.auth.PropertiesCredentials;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDB;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBAsyncClient;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
+import com.amazonaws.services.dynamodbv2.model.AttributeDefinition;
+import com.amazonaws.services.dynamodbv2.model.CreateTableRequest;
+import com.amazonaws.services.dynamodbv2.model.DescribeTableRequest;
+import com.amazonaws.services.dynamodbv2.model.KeySchemaElement;
+import com.amazonaws.services.dynamodbv2.model.ProvisionedThroughput;
+import com.amazonaws.services.dynamodbv2.model.ResourceInUseException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
+import com.amazonaws.services.dynamodbv2.model.TableStatus;
+
+public class DynamoDBUtils {
+
+  public enum DynamoDBType {
+    DYNAMO("native"), AVRO("avro");
+    private String value;
+
+    DynamoDBType(String val) {
+      this.value = val;
+    }
+
+    @Override
+    public String toString() {
+      return this.value;
+    }
+  }
+
+  public static final String DYNAMO_KEY_HASHRANGE = "hashrange";
+  public static final String DYNAMO_KEY_HASHR = "hash";
+
+  /** AWS Credential file name. */
+  public static final String AWS_CREDENTIALS_PROPERTIES = "awscredentials.properties";
+
+  /** Name of the cloud database provider. */
+  public static final String WS_PROVIDER = "amazon.web.services";
+
+  /** Parameter to decide what type of Amazon DynamoDB client to use */
+  public static final String CLI_TYP_PROP = "gora.dynamodb.client";
+
+  /** Parameter to decide where the data store will make its computations */
+  public static final String ENDPOINT_PROP = "gora.dynamodb.endpoint";
+
+  /** Parameter to decide which schema will be used */
+  public static final String PREF_SCH_NAME = "preferred.schema.name";
+
+  /**
+   * Parameter to decide how reads will be made i.e. using strong consistency or
+   * eventual consistency.
+   */
+  public static final String CONSISTENCY_READS = "gora.dynamodb.consistent.reads";
+  public static final String CONSISTENCY_READS_TRUE = "true";
+
+  /**
+   * Parameter to decide how serialization will be made i.e. using Dynamodb's or
+   * Avro serialization.
+   */
+  public static final String SERIALIZATION_TYPE = "gora.dynamodb.serialization.type";
+  public static final String DYNAMO_SERIALIZATION = "dynamo";
+  public static final String AVRO_SERIALIZATION = "avro";
+
+  /** DynamoDB client types. */
+  public static final String SYNC_CLIENT_PROP = "sync";
+  public static final String ASYNC_CLIENT_PROP = "async";
+
+  /** The mapping file to create the tables from. */
+  public static final String MAPPING_FILE = "gora-dynamodb-mapping.xml";
+
+  /** Default times to wait while requests are performed. */
+  public static long WAIT_TIME = 10L * 60L * 1000L;
+  public static long SLEEP_TIME = 1000L * 20L;
+  public static long SLEEP_DELETE_TIME = 1000L * 10L;
+
+  public static final Logger LOG = LoggerFactory.getLogger(DynamoDBUtils.class);
+
+  /**
+   * Method to create the specific client to be used
+   * 
+   * @param clientType
+   * @param credentials
+   * @return
+   */
+  public static AmazonDynamoDB getClient(String clientType,
+      AWSCredentials credentials) {
+    if (clientType.equals(SYNC_CLIENT_PROP))
+      return new AmazonDynamoDBClient(credentials);
+    if (clientType.equals(ASYNC_CLIENT_PROP))
+      return new AmazonDynamoDBAsyncClient(credentials);
+    return null;
+  }
+
+  /**
+   * Creates the AWSCredentials object based on the properties file.
+   * 
+   * @param awsCredentialsProperties
+   * @throws FileNotFoundException
+   * @throws IllegalArgumentException
+   * @throws IOException
+   */
+  public static PropertiesCredentials getCredentials(Class<?> clazz) {
+    PropertiesCredentials awsCredentials = null;
+    try {
+      InputStream awsCredInpStr = clazz.getClassLoader().getResourceAsStream(
+          AWS_CREDENTIALS_PROPERTIES);
+      if (awsCredInpStr == null)
+        LOG.error("AWS Credentials File was not found on the classpath!");
+      awsCredentials = new PropertiesCredentials(awsCredInpStr);
+    } catch (IOException e) {
+      LOG.error("Error loading AWS Credentials File from the classpath!", e.getMessage());
+      throw new RuntimeException(e);
+    }
+    return awsCredentials;
+  }
+
+  /**
+   * Executes a create table request using the DynamoDB client and waits the
+   * default time until it's been created.
+   * 
+   * @param awsClient
+   * @param keySchema
+   * @param tableName
+   * @param proThrou
+   */
+  public static void executeCreateTableRequest(AmazonDynamoDB awsClient, String tableName,
+      ArrayList<KeySchemaElement> keySchema, Map<String, String> attrs, ProvisionedThroughput proThrou) {
+    CreateTableRequest createTableRequest = buildCreateTableRequest(tableName,
+        keySchema, proThrou, attrs);
+    // use the client to perform the request
+    try {
+      awsClient.createTable(createTableRequest).getTableDescription();
+      // wait for table to become active
+      waitForTableToBecomeAvailable(awsClient, tableName);
+    } catch (ResourceInUseException ex) {
+      LOG.warn("Table '{}' already exists.", tableName);
+    } finally {
+      LOG.info("Table '{}' is available.", tableName);
+    }
+  }
+
+  /**
+   * Builds the necessary requests to create tables
+   * 
+   * @param tableName
+   * @param keySchema
+   * @param proThrou
+   * @param attrs 
+   * @return
+   */
+  public static CreateTableRequest buildCreateTableRequest(String tableName,
+      ArrayList<KeySchemaElement> keySchema, ProvisionedThroughput proThrou, Map<String, String> attrs) {
+    CreateTableRequest createTableRequest = new CreateTableRequest();
+    createTableRequest.setTableName(tableName);
+    createTableRequest.setKeySchema(keySchema);
+    ArrayList<AttributeDefinition> attributeDefinitions = new ArrayList<AttributeDefinition>();
+    for (KeySchemaElement kEle : keySchema) {
+      AttributeDefinition attrDef = new AttributeDefinition();
+      attrDef.setAttributeName(kEle.getAttributeName());
+      attrDef.setAttributeType(attrs.get(kEle.getAttributeName()));
+      attributeDefinitions.add(attrDef);
+    }
+    createTableRequest.setAttributeDefinitions(attributeDefinitions);
+    createTableRequest.setProvisionedThroughput(proThrou);
+    return createTableRequest;
+  }
+
+  /**
+   * Waits up to 6 minutes to confirm if a table has been created or not
+   * 
+   * @param awsClient
+   * @param tableName
+   */
+  public static void waitForTableToBecomeAvailable(AmazonDynamoDB awsClient,
+      String tableName) {
+    LOG.debug("Waiting for {} to become available", tableName);
+    long startTime = System.currentTimeMillis();
+    long endTime = startTime + WAIT_TIME;
+    while (System.currentTimeMillis() < endTime) {
+      try {
+        Thread.sleep(SLEEP_TIME);
+      } catch (Exception e) {
+      }
+      try {
+        DescribeTableRequest request = new DescribeTableRequest()
+            .withTableName(tableName);
+        TableDescription tableDescription = awsClient.describeTable(request)
+            .getTable();
+        String tableStatus = tableDescription.getTableStatus();
+        LOG.debug("{} - current state: {}", tableName, tableStatus);
+        if (tableStatus.equals(TableStatus.ACTIVE.toString()))
+          return;
+      } catch (AmazonServiceException ase) {
+        if (ase.getErrorCode().equalsIgnoreCase("ResourceNotFoundException") == false)
+          throw ase;
+      }
+    }
+    throw new RuntimeException("Table " + tableName + " never became active");
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/IDynamoDB.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/IDynamoDB.java b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/IDynamoDB.java
new file mode 100644
index 0000000..7aaf02f
--- /dev/null
+++ b/gora-dynamodb/src/main/java/org/apache/gora/dynamodb/store/IDynamoDB.java
@@ -0,0 +1,31 @@
+/*
+ * 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 org.apache.gora.persistency.Persistent;
+import org.apache.gora.store.DataStore;
+
+public interface IDynamoDB<K, T extends Persistent> extends DataStore<K, T> {
+
+  /**
+   * Sets the handler to the main DynamoDB
+   * @param DynamoDBStore handler to main DynamoDB
+   */
+  public abstract void setDynamoDBStoreHandler(DynamoDBStore<K, T> dynamoHandler);
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/test/conf/gora-dynamodb-mapping.xml
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/test/conf/gora-dynamodb-mapping.xml b/gora-dynamodb/src/test/conf/gora-dynamodb-mapping.xml
index 77dfbe0..a8fd080 100644
--- a/gora-dynamodb/src/test/conf/gora-dynamodb-mapping.xml
+++ b/gora-dynamodb/src/test/conf/gora-dynamodb-mapping.xml
@@ -19,17 +19,17 @@
 
 <gora-otd>
 
-  <table name="person" readcunit="5" writecunit="5"> <!-- optional descriptors for tables -->
-  	<key name="ssn" type="hash" att-type="S"/>
-  	<key name="date" type="hashrange" att-type="S"/>
+  <table name="Person" readcunit="1" writecunit="1" package="org.apache.gora.dynamodb.example.generated"> <!-- optional descriptors for tables -->
+    <attribute name="ssn" type="N" key="hash"/>
+    <attribute name="date" type="S" key="hashrange"/>
     <attribute name="firstName" type="S"/>
     <attribute name="lastName" type="S"/>
     <attribute name="salary" type="N"/>
     <attribute name="visitedplaces" type="SS"/>
   </table>
 
-  <table name="webpage" readcunit="5" writecunit="5">
-  	<key name="id" type="hash" att-type="S"/>
+  <table name="Webpage" readcunit="1" writecunit="1" package="org.apache.gora.dynamodb.example.generated">
+    <attribute name="id" type="S" key="hash"/>
     <attribute name="common" type="S"/>
     <attribute name="content" type="S"/>
     <attribute name="parsedContent" type="S"/>

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/test/conf/gora.properties
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/test/conf/gora.properties b/gora-dynamodb/src/test/conf/gora.properties
index 6f79ec3..75c98b4 100644
--- a/gora-dynamodb/src/test/conf/gora.properties
+++ b/gora-dynamodb/src/test/conf/gora.properties
@@ -30,4 +30,7 @@ gora.dynamodb.endpoint=http://dynamodb.us-east-1.amazonaws.com/
 #Asia Pacific (Tokyo) Region	        dynamodb.ap-northeast-1.amazonaws.com	HTTP and HTTPS
 #Asia Pacific (Singapore) Region	dynamodb.ap-southeast-1.amazonaws.com	HTTP and HTTPS
 
+#Data store serialization type. It could be 'dynamo' or 'avro'
+gora.dynamodb.serialization.type=dynamo
+
 

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/GoraDynamoDBTestDriver.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/GoraDynamoDBTestDriver.java b/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/GoraDynamoDBTestDriver.java
index d4cf8eb..aa86170 100644
--- a/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/GoraDynamoDBTestDriver.java
+++ b/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/GoraDynamoDBTestDriver.java
@@ -23,9 +23,9 @@ import java.io.FileNotFoundException;
 import java.io.IOException;
 
 import org.apache.gora.GoraTestDriver;
+import org.apache.gora.dynamodb.example.generated.Person;
 import org.apache.gora.dynamodb.query.DynamoDBKey;
 import org.apache.gora.dynamodb.store.DynamoDBStore;
-import org.apache.gora.examples.generated.person;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.store.ws.impl.WSDataStoreFactory;
@@ -33,10 +33,10 @@ import org.apache.gora.util.GoraException;
 
 import com.amazonaws.auth.AWSCredentials;
 import com.amazonaws.auth.PropertiesCredentials;
-import com.amazonaws.services.dynamodb.AmazonDynamoDBClient;
-import com.amazonaws.services.dynamodb.model.DescribeTableRequest;
-import com.amazonaws.services.dynamodb.model.ResourceNotFoundException;
-import com.amazonaws.services.dynamodb.model.TableDescription;
+import com.amazonaws.services.dynamodbv2.AmazonDynamoDBClient;
+import com.amazonaws.services.dynamodbv2.model.DescribeTableRequest;
+import com.amazonaws.services.dynamodbv2.model.ResourceNotFoundException;
+import com.amazonaws.services.dynamodbv2.model.TableDescription;
 
 /**
  * Helper class for third part tests using gora-dynamodb backend. 
@@ -47,28 +47,28 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
   /**
    * Data store to be used within the test driver
    */
-  private static DynamoDBStore<DynamoDBKey,person> personStore;
-  
+  private static DataStore<DynamoDBKey, Person> personStore;
+
   /**
    * DynamoDB client to be used from the test driver
    */
   static AmazonDynamoDBClient dynamoDBClient;
-  
+
   /**
    * Credentials file name
    */
   static String awsCredentialsFile = "AwsCredentials.properties";
-  
+
   /**
    * Test credential paths
    */
   static String awsCredentialsPath = "target/test-classes/";
-  
+
   /**
    * Authentication object
    */
   protected Object auth;
-  
+
   /**
    * Default constructor
    */
@@ -80,11 +80,11 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
       credentials = new PropertiesCredentials(file);
       auth = credentials;
     } catch (FileNotFoundException e) {
-      e.printStackTrace();
+      throw new RuntimeException(e);
     } catch (IllegalArgumentException e) {
-      e.printStackTrace();
+      throw new IllegalArgumentException(e);
     } catch (IOException e) {
-      e.printStackTrace();
+      throw new RuntimeException(e);
     }
   }
 
@@ -97,7 +97,7 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
     log.info("Initializing DynamoDB.");
     createDataStore();
   }
-  
+
   /**
    * Sets up the data store by creating the schema
    */
@@ -105,45 +105,48 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
   public void setUp() throws Exception {
     personStore.createSchema();
   }
-  
+
   /**
    * Creates the DynamoDB store and returns an specific object
+   * 
    * @return
    * @throws IOException
    */
   @SuppressWarnings("unchecked")
-  protected DataStore<DynamoDBKey, person> createDataStore() throws IOException {
-    if(personStore == null)
-      personStore = WSDataStoreFactory.createDataStore(DynamoDBStore.class, 
-        DynamoDBKey.class,person.class, auth);
-      return personStore;
+  protected DataStore<DynamoDBKey, Person> createDataStore()
+      throws IOException {
+    if (personStore == null)
+      personStore = WSDataStoreFactory.createDataStore(DynamoDBStore.class,
+          DynamoDBKey.class, Person.class, auth);
+    return personStore;
   }
-  
+
   /**
    * Creates the DynamoDB store but returns a generic object
    */
   @SuppressWarnings("unchecked")
-  public<K, T extends Persistent> DataStore<K,T>
-    createDataStore(Class<K> keyClass, Class<T> persistentClass) throws GoraException {
-      personStore = (DynamoDBStore<DynamoDBKey, person>) WSDataStoreFactory.createDataStore(
-        (Class<? extends DataStore<K,T>>)dataStoreClass, keyClass, persistentClass, auth);
-      dataStores.add(personStore);
+  public <K, T extends Persistent> DataStore<K, T> createDataStore(
+      Class<K> keyClass, Class<T> persistentClass) throws GoraException {
+    personStore = (DynamoDBStore<DynamoDBKey, Person>) WSDataStoreFactory
+        .createDataStore((Class<? extends DataStore<K, T>>) dataStoreClass,
+            keyClass, persistentClass, auth);
+    dataStores.add(personStore);
     return (DataStore<K, T>) personStore;
   }
-  
+
   /**
    * Gets or create the DynamoDB data store
+   * 
    * @return
    */
-  public DataStore<DynamoDBKey, person> getDataStore(){
+  public DataStore<DynamoDBKey, Person> getDataStore(){
     try {
       if(personStore != null)
         return personStore;
       else
         return createDataStore();
     } catch (IOException e) {
-      e.printStackTrace();
-      return null;
+      throw new RuntimeException(e);
     }
   }
 
@@ -154,7 +157,7 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
   public void tearDownClass() throws Exception {
     log.info("Finished DynamoDB driver.");
   }
-  
+
   /**
    * Tears down objects created
    */
@@ -162,39 +165,45 @@ public class GoraDynamoDBTestDriver extends GoraTestDriver {
   public void tearDown() throws Exception{
     super.tearDown();
   }
-  
+
   /**
    * Gets authentication object
+   * 
    * @return
    */
   public Object getAuth() {
     return auth;
   }
-  
+
   /**
    * Gets DynamoDBClient to be used
+   * 
    * @return
    */
   public AmazonDynamoDBClient getDynamoDBClient() {
     return dynamoDBClient;
   }
-  
+
   /**
    * Checks if a resource exists or not
-   * @param tableName	Table name to be checked
+   * 
+   * @param tableName
+   *          Table name to be checked
    * @return
    */
   public TableDescription checkResource(String tableName){
     TableDescription tableDescription = null;
-  
+
     try{
-      DescribeTableRequest describeTableRequest = new DescribeTableRequest().withTableName(tableName);
-      tableDescription = dynamoDBClient.describeTable(describeTableRequest).getTable();
+      DescribeTableRequest describeTableRequest = new DescribeTableRequest()
+          .withTableName(tableName);
+      tableDescription = dynamoDBClient.describeTable(describeTableRequest)
+          .getTable();
     }
     catch(ResourceNotFoundException e){
       tableDescription = null;
     }
-      
+
     return tableDescription;
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/655cd3aa/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBNativeStore.java
----------------------------------------------------------------------
diff --git a/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBNativeStore.java b/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBNativeStore.java
new file mode 100644
index 0000000..68e8776
--- /dev/null
+++ b/gora-dynamodb/src/test/java/org/apache/gora/dynamodb/TestDynamoDBNativeStore.java
@@ -0,0 +1,409 @@
+/*
+ * 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;
+
+import static junit.framework.Assert.assertFalse;
+import static junit.framework.Assert.assertTrue;
+
+import java.io.IOException;
+import java.util.HashSet;
+
+import org.apache.gora.dynamodb.example.generated.Person;
+import org.apache.gora.dynamodb.query.DynamoDBKey;
+import org.apache.gora.dynamodb.query.DynamoDBQuery;
+import org.apache.gora.query.Query;
+import org.apache.gora.query.Result;
+import org.apache.gora.store.DataStore;
+import org.apache.gora.store.WSDataStoreTestBase;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.amazonaws.services.dynamodbv2.model.ComparisonOperator;
+
+/**
+ * Test case for DynamoDBStore.
+ */
+public class TestDynamoDBNativeStore extends
+WSDataStoreTestBase<DynamoDBKey, Person> {
+
+  public static final Logger log = LoggerFactory
+      .getLogger(TestDynamoDBNativeStore.class);
+
+  static {
+    setTestDriver(new GoraDynamoDBTestDriver());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    setPersistentKeyClass(DynamoDBKey.class);
+    setPersistentValClass(Person.class);
+    super.setUp();
+  }
+
+  public GoraDynamoDBTestDriver getTestDriver() {
+    return (GoraDynamoDBTestDriver) testDriver;
+  }
+
+  // ============================================================================
+  // We need to skip the following tests for a while until we fix some issues..
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testQueryStartKey() throws IOException {
+    log.info("test method: TestQueryStartKey SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testQueryEndKey() throws IOException {
+    log.info("test method: TestQueryEndKey SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testDeleteByQueryFields() throws IOException {
+    log.info("test method: TestDeleteByQueryFields SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testNewInstance() throws IOException, Exception {
+    log.info("test method: TestNewInstance SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testAutoCreateSchema() throws Exception {
+    log.info("test method: TestAutoCreateSchema SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testTruncateSchema() throws Exception {
+    log.info("test method: TestTruncateSchema SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testPutNested() throws IOException, Exception {
+    log.info("test method: TestPutNested SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testPutArray() throws IOException, Exception {
+    log.info("test method: TestPutArray SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testPutBytes() throws IOException, Exception {
+    log.info("test method: TestPutBytes SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testPutMap() throws IOException, Exception {
+    log.info("test method: TestPutMap SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testEmptyUpdate() throws IOException, Exception {
+    log.info("test method: TestEmptyUpdate SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testDeleteSchema() throws IOException, Exception {
+    log.info("test method: TestDeleteSchema SKIPPED.");
+  }
+
+  @Ignore("Needs to be skipped for a while until some issues are fixed")
+  @Override
+  public void testGetWithFields() throws IOException, Exception {
+    log.info("test method: TestGetWithFields SKIPPED.");
+  }
+
+  // ==========================================================================
+
+  /**
+   * Tests deleting items using a query
+   */
+  @Override
+  public void assertTestDeleteByQueryDataStore() {
+    try {
+      log.info("test method: TestDeleteByQuery using DynamoDB store.");
+      DynamoDBKey<Long, String> dKey = new DynamoDBKey<>();
+      dKey.setHashKey(100L);
+      dKey.setRangeKey("10/10/1880");
+      Person p1 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "John", "Doe", "Peru", "Brazil", "Ecuador");
+      dataStore.put(dKey, p1);
+      dKey.setRangeKey("11/10/1707");
+      Person p2 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "Juan", "Perez", "Germany", "USA", "Scotland");
+      dataStore.put(dKey, p2);
+      DynamoDBQuery.setScanCompOp(ComparisonOperator.LE);
+      DynamoDBQuery.setType(DynamoDBQuery.SCAN_QUERY);
+      Query<DynamoDBKey, Person> query = new DynamoDBQuery<DynamoDBKey, Person>();
+      query.setKey(dKey);
+      log.info("Number of records deleted: " + dataStore.deleteByQuery(query));
+    } catch (Exception e) {
+      log.error("Error while running test: TestDeleteByQuery", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Tests updating a specific item
+   */
+  @Override
+  public void assertTestUpdateDataStore() {
+    try {
+      log.info("test method: TestUpdate using DynamoDB store.");
+      DynamoDBKey<Long, String> dKey = new DynamoDBKey<>();
+      dKey.setHashKey(13L);
+      dKey.setRangeKey("10/10/1880");
+      Person p1 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
+      dataStore.put(dKey, p1);
+      p1.setFirstName("Ataucuri");
+      dataStore.put(dKey, p1);
+    } catch (Exception e) {
+      log.error("error in test method: testUpdate.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to test deleting a schema
+   */
+  @Override
+  public void assertDeleteSchema() {
+    try {
+      log.info("test method: TestDeleteSchema using DynamoDB store.");
+      dataStore.deleteSchema();
+    } catch (Exception e) {
+      log.error("error in test method: testDeleteSchema.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to verify if a schema exists or not
+   */
+  @Override
+  public void assertSchemaExists(String schemaName) throws Exception {
+    log.info("test method: TestSchemaExists using DynamoDB store.");
+    assertTrue(dataStore.schemaExists());
+  }
+
+  /**
+   * Method to put items into the data store
+   */
+  @Override
+  public void assertPut() {
+    try {
+      log.info("test method: TestPut using DynamoDB store.");
+      DynamoDBKey<Long, String> dKey = new DynamoDBKey<>();
+      dKey.setHashKey(12L);
+      dKey.setRangeKey("10/10/1880");
+      Person p1 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
+      dataStore.put(dKey, p1);
+      dKey.setRangeKey("11/10/1707");
+      Person p2 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "William", "Wallace", "Germany", "USA", "Scotland");
+      dataStore.put(dKey, p2);
+    } catch (Exception e) {
+      log.error("error in test method: testPut.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to query the data store
+   */
+  @Override
+  public void assertTestQueryDataStore() {
+    log.info("test method: testQuery using DynamoDB store.");
+    try {
+      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
+      dKey.setHashKey("Peru");
+      DynamoDBQuery.setScanCompOp(ComparisonOperator.LE);
+      DynamoDBQuery.setType(DynamoDBQuery.SCAN_QUERY);
+      Query<DynamoDBKey, Person> query = new DynamoDBQuery<DynamoDBKey, Person>();
+      query.setKey(dKey);
+      Result<DynamoDBKey, Person> queryResult = dataStore.execute(query);
+      processQueryResult(queryResult);
+    } catch (Exception e) {
+      log.error("error in test method: testQuery.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to query items into the data store
+   */
+  @Override
+  public void assertTestQueryKeyRange() {
+    log.info("test method: testQueryKeyRange using specific data store.");
+    try {
+      DynamoDBKey<String, String> dKey = new DynamoDBKey<String, String>();
+      DynamoDBKey<String, String> startKey = new DynamoDBKey<String, String>();
+      DynamoDBKey<String, String> endKey = new DynamoDBKey<String, String>();
+      dKey.setHashKey("Peru");
+      startKey.setRangeKey("01/01/1700");
+      endKey.setRangeKey("31/12/1900");
+      DynamoDBQuery.setRangeCompOp(ComparisonOperator.BETWEEN);
+      DynamoDBQuery.setType(DynamoDBQuery.RANGE_QUERY);
+      Query<DynamoDBKey, Person> query = new DynamoDBQuery<DynamoDBKey, Person>();
+      query.setKey(dKey);
+      query.setStartKey(startKey);
+      query.setEndKey(endKey);
+      Result<DynamoDBKey, Person> queryResult = dataStore.execute(query);
+      processQueryResult(queryResult);
+    } catch (Exception e) {
+      log.error("error in test method: testQueryKeyRange.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to get an specific object using a key
+   */
+  @Override
+  public void assertTestGetDataStore() {
+    log.info("test method: testGet using specific data store.");
+    try {
+      DynamoDBKey<Long, String> dKey = new DynamoDBKey<>();
+      dKey.setHashKey(11L);
+      dKey.setRangeKey("10/10/1999");
+      // insert item
+      Person p1 = buildPerson(dKey.getHashKey(), dKey.getRangeKey().toString(),
+          "Inca", "Atahualpa", "Peru", "Brazil", "Ecuador");
+      dataStore.put(dKey, p1);
+      // get item
+      Person p2 = dataStore.get(dKey);
+      printPersonInfo(p2);
+    } catch (Exception e) {
+      log.error("error in test method: testGetDataStore.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to delete items into the data store
+   */
+  @Override
+  public void assertTestDeleteDataStore() {
+    log.info("test method: testDelete by key");
+    try {
+      DynamoDBKey<Long, String> dKey = new DynamoDBKey<Long, String>();
+      dKey.setHashKey(10L);
+      dKey.setRangeKey("10/10/1985");
+      Person p1 = new Person();
+      p1.setHashKey(dKey.getHashKey());
+      p1.setRangeKey(dKey.getRangeKey());
+      p1.setFirstName("Joao");
+      p1.setLastName("Velasco");
+      dataStore.put(dKey, p1);
+      assertTrue(dataStore.delete(dKey));
+      dKey.setRangeKey("10/10/1000");
+      assertFalse(dataStore.delete(dKey));
+    } catch (Exception e) {
+      log.error("error in test method: testDeleteDataStore.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to create the data store
+   */
+  @Override
+  protected DataStore<DynamoDBKey, Person> createDataStore() {
+    log.info("Creating DynamoDB data store.");
+    try {
+      dataStore = getTestDriver().getDataStore();
+      dataStore.createSchema();
+    } catch (Exception e) {
+      log.error("error while creating DynamoDB data store", e.getMessage());
+      throw new RuntimeException(e);
+    }
+    return dataStore;
+  }
+
+  /**
+   * Processes query results from an query execution
+   * 
+   * @param pQueryResult
+   */
+  private void processQueryResult(Result<DynamoDBKey, Person> pQueryResult) {
+    try {
+      log.debug("Processing tests results.");
+      while (pQueryResult.next())
+        printPersonInfo(pQueryResult.get());
+    } catch (IOException e) {
+      log.error("error while processing tests results.", e.getMessage());
+      throw new RuntimeException(e);
+    } catch (Exception e) {
+      log.error("error while processing tests results.", e.getMessage());
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Method to generate persisten objects
+   * 
+   * @param key
+   * @param pRangeKey
+   * @param pFirstName
+   * @param pLastName
+   * @param places
+   * @return
+   */
+  private Person buildPerson(Long key, String pRangeKey, String pFirstName,
+      String pLastName, String... places) {
+    Person newPerson = new Person();
+    newPerson.setRangeKey(pRangeKey);
+    newPerson.setHashKey(key);
+    newPerson.setFirstName(pFirstName);
+    newPerson.setLastName(pLastName);
+    newPerson.setVisitedplaces(new HashSet<String>());
+    for (String place : places)
+      newPerson.getVisitedplaces().add(place);
+    return newPerson;
+  }
+
+  /**
+   * Method to print the object returned from Get method
+   * 
+   * @param pPerson
+   */
+  private void printPersonInfo(Person pPerson) {
+    log.info("Origin:\t" + pPerson.getHashKey() + "\n Birthday:\t"
+        + pPerson.getRangeKey() + "\n FirstName:" + pPerson.getFirstName()
+        + "\n LastName:" + pPerson.getLastName() + "\n Visited Places:");
+    for (String place : pPerson.getVisitedplaces())
+      log.info("\t" + place);
+  }
+
+}
\ No newline at end of file