You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by al...@apache.org on 2018/02/27 12:07:54 UTC

[2/5] gora git commit: GORA-530 : Reinstated exception throwing in DataStore and Query

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/avro/store/AvroStore.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/avro/store/AvroStore.java b/gora-core/src/main/java/org/apache/gora/avro/store/AvroStore.java
index f1f79c9..570d5ec 100644
--- a/gora-core/src/main/java/org/apache/gora/avro/store/AvroStore.java
+++ b/gora-core/src/main/java/org/apache/gora/avro/store/AvroStore.java
@@ -39,10 +39,10 @@ import org.apache.gora.query.Result;
 import org.apache.gora.query.impl.FileSplitPartitionQuery;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.impl.FileBackedDataStoreBase;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.OperationNotSupportedException;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -78,7 +78,7 @@ extends FileBackedDataStoreBase<K, T> implements Configurable {
 
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-          Properties properties) {
+          Properties properties) throws GoraException {
     super.initialize(keyClass, persistentClass, properties);
 
     if(properties != null && this.codecType == null) {
@@ -123,12 +123,12 @@ extends FileBackedDataStoreBase<K, T> implements Configurable {
   }
 
   @Override
-  public boolean delete(K key) {
+  public boolean delete(K key) throws GoraException {
     throw new OperationNotSupportedException("delete is not supported for AvroStore");
   }
 
   @Override
-  public long deleteByQuery(Query<K, T> query) {
+  public long deleteByQuery(Query<K, T> query) throws GoraException {
     throw new OperationNotSupportedException("delete is not supported for AvroStore");
   }
 
@@ -152,18 +152,19 @@ extends FileBackedDataStoreBase<K, T> implements Configurable {
   }
 
   @Override
-  public void flush() {
+  public void flush() throws GoraException {
     try{
       super.flush();
       if(encoder != null)
         encoder.flush();
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public T get(K key, String[] fields) {
+  public T get(K key, String[] fields) throws GoraException {
     throw new OperationNotSupportedException();
   }
 
@@ -173,11 +174,12 @@ extends FileBackedDataStoreBase<K, T> implements Configurable {
   }
 
   @Override
-  public void put(K key, T obj) {
+  public void put(K key, T obj) throws GoraException {
     try{
       getDatumWriter().write(obj, getEncoder());
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -246,12 +248,12 @@ extends FileBackedDataStoreBase<K, T> implements Configurable {
   }
 
   @Override
-  public void write(DataOutput out) {
+  public void write(DataOutput out) throws IOException {
     super.write(out);
   }
 
   @Override
-  public void readFields(DataInput in) {
+  public void readFields(DataInput in) throws IOException {
     super.readFields(in);
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/avro/store/DataFileAvroStore.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/avro/store/DataFileAvroStore.java b/gora-core/src/main/java/org/apache/gora/avro/store/DataFileAvroStore.java
index afbdad1..0e42d71 100644
--- a/gora-core/src/main/java/org/apache/gora/avro/store/DataFileAvroStore.java
+++ b/gora-core/src/main/java/org/apache/gora/avro/store/DataFileAvroStore.java
@@ -28,6 +28,7 @@ import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
 import org.apache.gora.query.impl.FileSplitPartitionQuery;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.OperationNotSupportedException;
 import org.apache.hadoop.fs.Path;
 
@@ -49,17 +50,18 @@ public class DataFileAvroStore<K, T extends PersistentBase> extends AvroStore<K,
   private DataFileWriter<T> writer;
   
   @Override
-  public T get(K key, String[] fields) {
+  public T get(K key, String[] fields) throws GoraException {
     throw new OperationNotSupportedException(
         "Avro DataFile's does not support indexed retrieval");
   }
 
   @Override
-  public void put(K key, T obj) {
+  public void put(K key, T obj) throws GoraException {
     try{
       getWriter().append(obj);
-    } catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -72,27 +74,17 @@ public class DataFileAvroStore<K, T extends PersistentBase> extends AvroStore<K,
   }
   
   @Override
-  protected Result<K, T> executeQuery(Query<K, T> query) {
-    try{
+  protected Result<K, T> executeQuery(Query<K, T> query) throws IOException {
       return new DataFileAvroResult<>(this, query
           , createReader(createFsInput()));
-    } catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
-      return null;
-    }
   }
  
   @Override
-  protected Result<K,T> executePartial(FileSplitPartitionQuery<K,T> query) {
-    try{
+  protected Result<K,T> executePartial(FileSplitPartitionQuery<K,T> query) throws IOException {
       FsInput fsInput = createFsInput();
       DataFileReader<T> reader = createReader(fsInput);
       return new DataFileAvroResult<>(this, query, reader, fsInput
           , query.getStart(), query.getLength());
-    } catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
-      return null;
-    }
   }
   
   private DataFileReader<T> createReader(FsInput fsInput) throws IOException {
@@ -105,14 +97,17 @@ public class DataFileAvroStore<K, T extends PersistentBase> extends AvroStore<K,
   }
   
   @Override
-  public void flush() {
+  public void flush() throws GoraException {
     try{
       super.flush();
       if(writer != null) {
         writer.flush();
       }
-    } catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/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 b67c7c1..06596d5 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
@@ -21,10 +21,10 @@ package org.apache.gora.memory.store;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 import java.util.NavigableMap;
-import java.util.concurrent.ConcurrentNavigableMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 
 import org.apache.avro.Schema.Field;
@@ -38,6 +38,7 @@ import org.apache.gora.query.impl.ResultBase;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.store.impl.DataStoreBase;
 import org.apache.gora.util.AvroUtils;
+import org.apache.gora.util.GoraException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -147,7 +148,7 @@ public class MemStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
    */
   @SuppressWarnings("unchecked")
   @Override
-  public Result<K, T> execute(Query<K, T> query) {
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
     K startKey = query.getStartKey();
     K endKey = query.getEndKey();
     if(startKey == null) {
@@ -163,14 +164,19 @@ public class MemStore<K, T extends PersistentBase> extends DataStoreBase<K, T> {
 
     //check if query.fields is null
     query.setFields(getFieldsToQuery(query.getFields()));
-    ConcurrentNavigableMap<K,T> submap = null;
-    try {
-      submap =  map.subMap(startKey, true, endKey, true);
-    } catch (NullPointerException npe){
-      LOG.info("Either startKey || endKey || startKey and endKey value(s) is null. "
-          + "No results will be returned for query to MemStore.");
-      return new MemResult<>(this, query, new ConcurrentSkipListMap<K, T>());
+    NavigableMap<K,T> submap = null;
+    if (startKey != null && endKey != null) {
+      try {
+        submap =  map.subMap(startKey, true, endKey, true);
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new GoraException(e);
+      }
+    } else {
+      // Empty
+      submap = Collections.emptyNavigableMap() ;
     }
+    
     return new MemResult<>(this, query, submap);
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java b/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
index 763cba1..1e9911a 100644
--- a/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
+++ b/gora-core/src/main/java/org/apache/gora/persistency/impl/BeanFactoryImpl.java
@@ -22,8 +22,11 @@ import java.lang.reflect.Constructor;
 
 import org.apache.gora.persistency.BeanFactory;
 import org.apache.gora.persistency.Persistent;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.ReflectionUtils;
 
+import com.esotericsoftware.minlog.Log;
+
 /**
  * A default implementation of the {@link BeanFactory} interface. Constructs
  * the keys by using reflection, {@link Persistent} objects by calling
@@ -57,7 +60,7 @@ public class BeanFactoryImpl<K, T extends Persistent> implements BeanFactory<K,
    * @param keyClass class of the keys
    * @param persistentClass class of the [{@link Persistent} objects to be stored
    */
-  public BeanFactoryImpl(Class<K> keyClass, Class<T> persistentClass) {
+  public BeanFactoryImpl(Class<K> keyClass, Class<T> persistentClass) throws GoraException {
     this.keyClass = keyClass;
     this.persistentClass = persistentClass;
     
@@ -68,7 +71,8 @@ public class BeanFactoryImpl<K, T extends Persistent> implements BeanFactory<K,
       }
       this.persistent = ReflectionUtils.newInstance(persistentClass);
     } catch (Exception ex) {
-      throw new RuntimeException(ex);
+      Log.error(ex.getMessage(), ex);
+      throw new GoraException(ex);
     }
     
     isKeyPersistent = Persistent.class.isAssignableFrom(keyClass);

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/query/Query.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/query/Query.java b/gora-core/src/main/java/org/apache/gora/query/Query.java
index 4001f65..447d4ec 100644
--- a/gora-core/src/main/java/org/apache/gora/query/Query.java
+++ b/gora-core/src/main/java/org/apache/gora/query/Query.java
@@ -21,6 +21,7 @@ package org.apache.gora.query;
 import org.apache.gora.filter.Filter;
 import org.apache.gora.persistency.Persistent;
 import org.apache.gora.store.DataStore;
+import org.apache.gora.util.GoraException;
 
 /**
  * A query to a data store to retrieve objects. Queries are constructed by 
@@ -46,7 +47,7 @@ public interface Query<K, T extends Persistent> {
    * Executes the Query on the DataStore and returns the results.
    * @return the {@link Result} for the query.
    */
-  Result<K, T> execute();
+  Result<K, T> execute() throws GoraException;
   
 //  /**
 //   * Compiles the query for performance and error checking. This 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java b/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
index 1f3908a..b614798 100644
--- a/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
+++ b/gora-core/src/main/java/org/apache/gora/query/impl/QueryBase.java
@@ -32,6 +32,7 @@ import org.apache.gora.query.Result;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.store.impl.DataStoreBase;
 import org.apache.gora.util.ClassLoadingUtils;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.IOUtils;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configurable; 
@@ -69,7 +70,7 @@ public abstract class QueryBase<K, T extends PersistentBase>
   }
 
   @Override
-  public Result<K,T> execute() {
+  public Result<K,T> execute() throws GoraException {
     return dataStore.execute(this);
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java b/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
index 4f9189f..c153e40 100644
--- a/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
+++ b/gora-core/src/main/java/org/apache/gora/query/ws/impl/QueryWSBase.java
@@ -26,6 +26,7 @@ import org.apache.gora.persistency.Persistent;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
 import org.apache.gora.store.DataStore;
+import org.apache.gora.util.GoraException;
 
 /**
  * Base class for Query implementations.
@@ -86,7 +87,7 @@ public abstract class QueryWSBase<K, T extends Persistent> implements Query<K,T>
   /**
    * Executes the query
    */
-  public Result<K,T> execute() {
+  public Result<K,T> execute() throws GoraException {
     //compile();
     return dataStore.execute(this);
   }

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/store/DataStore.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/store/DataStore.java b/gora-core/src/main/java/org/apache/gora/store/DataStore.java
index 55c71b9..b36476d 100644
--- a/gora-core/src/main/java/org/apache/gora/store/DataStore.java
+++ b/gora-core/src/main/java/org/apache/gora/store/DataStore.java
@@ -26,6 +26,7 @@ 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.util.GoraException;
 
 /**
  * DataStore handles actual object persistence. Objects can be persisted,
@@ -52,7 +53,7 @@ public interface DataStore<K, T extends Persistent> {
    * @param persistentClass the class of the persistent objects
    * @param properties extra metadata
    */
-  void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties);
+  void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) throws GoraException;
 
   /**
    * Sets the class of the keys
@@ -90,26 +91,26 @@ public interface DataStore<K, T extends Persistent> {
    * or the underlying data model does not support
    * or need this operation, the operation is ignored.
    */
-  void createSchema();
+  void createSchema() throws GoraException;
 
   /**
    * Deletes the underlying schema or table (or similar) in the datastore
    * that holds the objects. This also deletes all the data associated with
    * the schema.
    */
-  void deleteSchema();
+  void deleteSchema() throws GoraException;
 
   /**
    * Deletes all the data associated with the schema, but keeps the
    * schema (table or similar) intact.
    */
-  void truncateSchema();
+  void truncateSchema() throws GoraException;
 
   /**
    * Returns whether the schema that holds the data exists in the datastore.
    * @return whether schema exists
    */
-  boolean schemaExists();
+  boolean schemaExists() throws GoraException;
 
   /**
    * Returns a new instance of the key object. If the object cannot be instantiated 
@@ -118,20 +119,20 @@ public interface DataStore<K, T extends Persistent> {
    * make sure that the key class has a no-arg constructor.   
    * @return a new instance of the key object.
    */
-  K newKey();
+  K newKey() throws GoraException;
 
   /**
    * Returns a new instance of the managed persistent object.
    * @return a new instance of the managed persistent object.
    */
-  T newPersistent();
+  T newPersistent() throws GoraException;
 
   /**
    * Returns the object corresponding to the given key fetching all the fields.
    * @param key the key of the object
    * @return the Object corresponding to the key or null if it cannot be found
    */
-  T get(K key);
+  T get(K key) throws GoraException;
 
   /**
    * Returns the object corresponding to the given key.
@@ -139,7 +140,7 @@ public interface DataStore<K, T extends Persistent> {
    * @param fields the fields required in the object. Pass null, to retrieve all fields
    * @return the Object corresponding to the key or null if it cannot be found
    */
-  T get(K key, String[] fields);
+  T get(K key, String[] fields) throws GoraException;
 
   /**
    * Inserts the persistent object with the given key. If an 
@@ -150,14 +151,14 @@ public interface DataStore<K, T extends Persistent> {
    * @param key the key of the object
    * @param obj the {@link Persistent} object
    */
-  void put(K key, T obj);
+  void put(K key, T obj) throws GoraException;
 
   /**
    * Deletes the object with the given key
    * @param key the key of the object
    * @return whether the object was successfully deleted
    */
-  boolean delete(K key);
+  boolean delete(K key) throws GoraException;
 
   /**
    * Deletes all the objects matching the query.
@@ -165,14 +166,14 @@ public interface DataStore<K, T extends Persistent> {
    * @param query matching records to this query will be deleted
    * @return number of deleted records
    */
-  long deleteByQuery(Query<K, T> query);
+  long deleteByQuery(Query<K, T> query) throws GoraException;
 
   /**
    * Executes the given query and returns the results.
    * @param query the query to execute.
    * @return the results as a {@link Result} object.
    */
-  Result<K, T> execute(Query<K, T> query);
+  Result<K, T> execute(Query<K, T> query) throws GoraException;
 
   /**
    * Constructs and returns a new Query.
@@ -199,7 +200,7 @@ public interface DataStore<K, T extends Persistent> {
    * until this moment.
    * See also the note on <a href="#visibility">visibility</a>.
    */
-  void flush();
+  void flush() throws GoraException;
 
   /**
    * Sets the {@link BeanFactory} to use by the DataStore.

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java b/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
index 6b21a67..cb0e5c5 100644
--- a/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
+++ b/gora-core/src/main/java/org/apache/gora/store/impl/DataStoreBase.java
@@ -30,7 +30,7 @@ import java.util.Properties;
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.specific.SpecificDatumReader;
-import org.apache.avro.specific.SpecificDatumWriter; 
+import org.apache.avro.specific.SpecificDatumWriter;
 import org.apache.commons.lang.builder.EqualsBuilder;
 import org.apache.gora.avro.store.AvroStore;
 import org.apache.gora.persistency.BeanFactory;
@@ -41,10 +41,11 @@ import org.apache.gora.store.DataStore;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.util.AvroUtils;
 import org.apache.gora.util.ClassLoadingUtils;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.StringUtils;
 import org.apache.gora.util.WritableUtils;
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.Writable;
 import org.slf4j.Logger;
@@ -84,7 +85,7 @@ public abstract class DataStoreBase<K, T extends PersistentBase>
 
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-          Properties properties) {
+          Properties properties) throws GoraException {
     setKeyClass(keyClass);
     setPersistentClass(persistentClass);
     if (this.beanFactory == null) {
@@ -122,22 +123,22 @@ public abstract class DataStoreBase<K, T extends PersistentBase>
   }
 
   @Override
-  public K newKey() {
+  public K newKey() throws GoraException {
     try {
       return beanFactory.newKey();
-    } catch (Exception ex) {
-      LOG.error(ex.getMessage(), ex);
-      return null;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public T newPersistent() {
+  public T newPersistent() throws GoraException {
     try {
       return beanFactory.newPersistent();
-    } catch (Exception ex) {
-      LOG.error(ex.getMessage(), ex);
-      return null;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -152,7 +153,7 @@ public abstract class DataStoreBase<K, T extends PersistentBase>
   }
 
   @Override
-  public T get(K key) {
+  public T get(K key) throws GoraException {
     return get(key, getFieldsToQuery(null));
   }
 
@@ -208,25 +209,22 @@ public abstract class DataStoreBase<K, T extends PersistentBase>
   }
 
   @SuppressWarnings("unchecked")
-  public void readFields(DataInput in) {
-    try {
+  public void readFields(DataInput in) throws IOException {
+    try {  
       Class<K> keyClass = (Class<K>) ClassLoadingUtils.loadClass(Text.readString(in));
       Class<T> persistentClass = (Class<T>)ClassLoadingUtils.loadClass(Text.readString(in));
       Properties props = WritableUtils.readProperties(in);
       initialize(keyClass, persistentClass, props);
-    } catch (ClassNotFoundException | IOException ex) {
-      LOG.error(ex.getMessage(), ex);
+    } catch (ClassNotFoundException e) {
+      LOG.error("ClassNotFoundException", e);
+      throw new IOException(e);
     }
   }
 
-  public void write(DataOutput out) {
-    try {
+  public void write(DataOutput out) throws IOException {
       Text.writeString(out, getKeyClass().getCanonicalName());
       Text.writeString(out, getPersistentClass().getCanonicalName());
       WritableUtils.writeProperties(out, properties);
-    } catch (IOException e) {
-      LOG.error(e.getMessage(), e);
-    }
   }
 
   @Override
@@ -244,7 +242,7 @@ public abstract class DataStoreBase<K, T extends PersistentBase>
 
   @Override
   /** Default implementation deletes and recreates the schema*/
-  public void truncateSchema() {
+  public void truncateSchema() throws GoraException {
     deleteSchema();
     createSchema();
   }

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/store/impl/FileBackedDataStoreBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/store/impl/FileBackedDataStoreBase.java b/gora-core/src/main/java/org/apache/gora/store/impl/FileBackedDataStoreBase.java
index 817a28f..a46b116 100644
--- a/gora-core/src/main/java/org/apache/gora/store/impl/FileBackedDataStoreBase.java
+++ b/gora-core/src/main/java/org/apache/gora/store/impl/FileBackedDataStoreBase.java
@@ -36,7 +36,7 @@ import org.apache.gora.query.Result;
 import org.apache.gora.query.impl.FileSplitPartitionQuery;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.FileBackedDataStore;
-import org.apache.gora.util.OperationNotSupportedException;
+import org.apache.gora.util.GoraException;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;
@@ -64,7 +64,7 @@ extends DataStoreBase<K, T> implements FileBackedDataStore<K, T> {
 
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-          Properties properties) {
+          Properties properties) throws GoraException {
     super.initialize(keyClass, persistentClass, properties);
     if(properties != null) {
       if(this.inputPath == null) {
@@ -166,24 +166,20 @@ extends DataStoreBase<K, T> implements FileBackedDataStore<K, T> {
   }
 
   @Override
-  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query){
+  public List<PartitionQuery<K, T>> getPartitions(Query<K, T> query) throws IOException {
     List<InputSplit> splits = null;
     List<PartitionQuery<K, T>> queries = null;
-    try{
-      splits = GoraMapReduceUtils.getSplits(getConf(), inputPath);
-      queries = new ArrayList<>(splits.size());
+    splits = GoraMapReduceUtils.getSplits(getConf(), inputPath);
+    queries = new ArrayList<>(splits.size());
 
-      for(InputSplit split : splits) {
-        queries.add(new FileSplitPartitionQuery<>(query, (FileSplit) split));
-      }
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    for(InputSplit split : splits) {
+      queries.add(new FileSplitPartitionQuery<>(query, (FileSplit) split));
     }
     return queries;
   }
 
   @Override
-  public Result<K, T> execute(Query<K, T> query) {
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
     Result<K, T> results = null;
     try{
       if(query instanceof FileSplitPartitionQuery) {
@@ -191,8 +187,11 @@ extends DataStoreBase<K, T> implements FileBackedDataStore<K, T> {
       } else {
         results = executeQuery(query);
       }
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
     return results;
   }
@@ -219,46 +218,43 @@ extends DataStoreBase<K, T> implements FileBackedDataStore<K, T> {
           throws IOException;
 
   @Override
-  public void flush() {
+  public void flush() throws GoraException {
     try{
       if(outputStream != null)
         outputStream.flush();
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public void createSchema() {
+  public void createSchema() throws GoraException {
   }
 
   @Override
-  public void deleteSchema() {
-    throw new OperationNotSupportedException("delete schema is not supported for " +
+  public void deleteSchema() throws GoraException {
+    throw new GoraException("delete schema is not supported for " +
             "file backed data stores");
   }
 
   @Override
-  public boolean schemaExists() {
+  public boolean schemaExists() throws GoraException {
     return true;
   }
 
   @Override
-  public void write(DataOutput out) {
-    try{
+  public void write(DataOutput out) throws IOException {
       super.write(out);
       org.apache.gora.util.IOUtils.writeNullFieldsInfo(out, inputPath, outputPath);
       if(inputPath != null)
         Text.writeString(out, inputPath);
       if(outputPath != null)
         Text.writeString(out, outputPath);
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
-    }
   }
 
   @Override
-  public void readFields(DataInput in) {
+  public void readFields(DataInput in) throws IOException {
     try{
       super.readFields(in);
       boolean[] nullFields = org.apache.gora.util.IOUtils.readNullFieldsInfo(in);

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSBackedDataStoreBase.java
----------------------------------------------------------------------
diff --git a/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSBackedDataStoreBase.java b/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSBackedDataStoreBase.java
index a62425a..bc806fa 100644
--- a/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSBackedDataStoreBase.java
+++ b/gora-core/src/main/java/org/apache/gora/store/ws/impl/WSBackedDataStoreBase.java
@@ -25,8 +25,8 @@ import org.apache.gora.persistency.Persistent;
 import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
 import org.apache.gora.store.WebServiceBackedDataStore;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.OperationNotSupportedException;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -44,7 +44,7 @@ public abstract class WSBackedDataStoreBase<K, T extends Persistent>
    * @throws IOException
    */
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-      Properties properties) {
+      Properties properties) throws GoraException {
     super.initialize(keyClass, persistentClass, properties);
   }
 
@@ -52,12 +52,12 @@ public abstract class WSBackedDataStoreBase<K, T extends Persistent>
   /**
    * Executes a query inside a web service backed data store
    */
-  public Result<K, T> execute(Query<K, T> query) {
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
     try {
       return executeQuery(query);
     } catch (IOException e) {
       LOG.error(e.getMessage());
-      throw new RuntimeException(e);
+      throw new GoraException(e);
     }
   }
 
@@ -72,21 +72,21 @@ public abstract class WSBackedDataStoreBase<K, T extends Persistent>
   /**
    * Flushes objects into the data store
    */
-  public void flush() {
+  public void flush() throws GoraException {
   }
 
   @Override
   /**
    * Creates schema into the data store
    */
-  public void createSchema() {
+  public void createSchema() throws GoraException {
   }
 
   @Override
   /**
    * Deletes schema from the data store
    */
-  public void deleteSchema() {
+  public void deleteSchema() throws GoraException {
     throw new OperationNotSupportedException("delete schema is not supported for " +
       "file backed data stores");
   }
@@ -95,7 +95,7 @@ public abstract class WSBackedDataStoreBase<K, T extends Persistent>
   /**
    * Verifies if a schema exists
    */
-  public boolean schemaExists() {
+  public boolean schemaExists() throws GoraException {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/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 494a09d..40483a9 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
@@ -23,6 +23,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.GoraException;
 import org.apache.gora.util.StringUtils;
 
 /**
@@ -72,7 +73,7 @@ implements DataStore<K, T>{
    */
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-      Properties properties) {
+      Properties properties) throws GoraException {
     setKeyClass(keyClass);
     setPersistentClass(persistentClass);
     autoCreateSchema = DataStoreFactory.getAutoCreateSchema(properties, this);
@@ -155,7 +156,7 @@ implements DataStore<K, T>{
 
   @Override
   /** Default implementation deletes and recreates the schema*/
-  public void truncateSchema() {
+  public void truncateSchema() throws GoraException {
     deleteSchema();
     createSchema();
   }

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/test/java/org/apache/gora/memory/store/MemStoreTest.java
----------------------------------------------------------------------
diff --git a/gora-core/src/test/java/org/apache/gora/memory/store/MemStoreTest.java b/gora-core/src/test/java/org/apache/gora/memory/store/MemStoreTest.java
index 3972cca..55aa8a2 100644
--- a/gora-core/src/test/java/org/apache/gora/memory/store/MemStoreTest.java
+++ b/gora-core/src/test/java/org/apache/gora/memory/store/MemStoreTest.java
@@ -17,18 +17,23 @@
  */
 package org.apache.gora.memory.store;
 
-import java.io.IOException;
+import static org.apache.gora.examples.WebPageDataCreator.SORTED_URLS;
+import static org.apache.gora.examples.WebPageDataCreator.URLS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeTrue;
 
 import org.apache.gora.examples.WebPageDataCreator;
-import org.apache.gora.examples.generated.Employee;
 import org.apache.gora.examples.generated.WebPage;
 import org.apache.gora.persistency.BeanFactory;
 import org.apache.gora.persistency.impl.BeanFactoryImpl;
 import org.apache.gora.query.Query;
 import org.apache.gora.store.DataStore;
-import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.DataStoreTestBase;
 import org.apache.gora.store.DataStoreTestUtil;
+import org.apache.gora.util.GoraException;
 import org.apache.hadoop.conf.Configuration;
 import org.junit.Before;
 import org.junit.Ignore;
@@ -36,15 +41,6 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.gora.examples.WebPageDataCreator.SORTED_URLS;
-import static org.apache.gora.examples.WebPageDataCreator.URLS;
-import static org.apache.gora.examples.WebPageDataCreator.URL_INDEXES;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assume.assumeTrue;
-
 /**
  * Testing class for all standard gora-memory functionality.
  * We extend {@link org.apache.gora.store.DataStoreTestBase} enabling us to run the entire base test
@@ -68,7 +64,7 @@ public class MemStoreTest extends DataStoreTestBase {
   }
 
   @Test
-  public void testGetMissingValue() {
+  public void testGetMissingValue() throws GoraException {
     DataStore<String, WebPage> store = new MemStore<>();
     WebPage nullWebPage = store.get("missing", new String[0]);
     assertNull(nullWebPage);

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/test/java/org/apache/gora/mock/store/MockDataStore.java
----------------------------------------------------------------------
diff --git a/gora-core/src/test/java/org/apache/gora/mock/store/MockDataStore.java b/gora-core/src/test/java/org/apache/gora/mock/store/MockDataStore.java
index 7a7bc93..afb8104 100644
--- a/gora-core/src/test/java/org/apache/gora/mock/store/MockDataStore.java
+++ b/gora-core/src/test/java/org/apache/gora/mock/store/MockDataStore.java
@@ -24,10 +24,13 @@ import java.util.List;
 
 import org.apache.gora.mock.persistency.MockPersistent;
 import org.apache.gora.mock.query.MockQuery;
+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.query.impl.PartitionQueryImpl;
+import org.apache.gora.query.impl.ResultBase;
+import org.apache.gora.store.DataStore;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.impl.DataStoreBase;
 import org.apache.gora.util.GoraException;
@@ -88,8 +91,8 @@ public class MockDataStore extends DataStoreBase<String, MockPersistent> {
   }
 
   @Override
-  public Result<String, MockPersistent> execute(Query<String, MockPersistent> query) {
-    return null;
+  public Result<String, MockPersistent> execute(Query<String, MockPersistent> query) throws GoraException {
+    return new MockResult<String, MockPersistent>(this, query);
   }
 
   @Override
@@ -141,4 +144,27 @@ public class MockDataStore extends DataStoreBase<String, MockPersistent> {
   @Override
   public void setPersistentClass(Class<MockPersistent> persistentClass) {
   }
+  
+  public static class MockResult<K, T extends PersistentBase> extends ResultBase<K, T> {
+    
+    public MockResult(DataStore<K, T> dataStore, Query<K, T> query) {
+      super(dataStore, query);
+    }
+    
+    @Override
+    public void close() throws IOException { }
+
+    @Override
+    public float getProgress() throws IOException {
+      return 0;
+    }
+
+    @Override
+    protected void clear() {  } //do not clear the object in the store
+
+    @Override
+    public boolean nextInner() throws IOException {
+      return false ;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
----------------------------------------------------------------------
diff --git a/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java b/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
index b9e58cd..b6e3d7c 100644
--- a/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
+++ b/gora-core/src/test/java/org/apache/gora/store/DataStoreTestUtil.java
@@ -56,6 +56,7 @@ import org.apache.gora.query.Query;
 import org.apache.gora.query.Result;
 import org.apache.gora.util.AvroUtils;
 import org.apache.gora.util.ByteUtils;
+import org.apache.gora.util.GoraException;
 import org.apache.gora.util.StringUtils;
 
 import org.slf4j.Logger;
@@ -1170,7 +1171,7 @@ public class DataStoreTestUtil {
     store.close();
   }
   
-  public static void testPutMixedMapTypes(DataStore<String, WebPage> store) {
+  public static void testPutMixedMapTypes(DataStore<String, WebPage> store) throws GoraException {
     WebPage webpage = createWebPage();
     webpage.getByteData().put(new Utf8("byteData"), ByteBuffer.wrap(ByteUtils.toBytes("hello map")));
     webpage.getStringData().put(new Utf8("stringData"), "hello map");

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-couchdb/src/main/java/org/apache/gora/couchdb/store/CouchDBStore.java
----------------------------------------------------------------------
diff --git a/gora-couchdb/src/main/java/org/apache/gora/couchdb/store/CouchDBStore.java b/gora-couchdb/src/main/java/org/apache/gora/couchdb/store/CouchDBStore.java
index 889d78f..8890ff3 100644
--- a/gora-couchdb/src/main/java/org/apache/gora/couchdb/store/CouchDBStore.java
+++ b/gora-couchdb/src/main/java/org/apache/gora/couchdb/store/CouchDBStore.java
@@ -18,7 +18,18 @@
 
 package org.apache.gora.couchdb.store;
 
-import com.google.common.primitives.Ints;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.util.Utf8;
@@ -38,8 +49,10 @@ import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.impl.DataStoreBase;
 import org.apache.gora.util.AvroUtils;
 import org.apache.gora.util.ClassLoadingUtils;
+import org.apache.gora.util.GoraException;
 import org.ektorp.CouchDbConnector;
 import org.ektorp.CouchDbInstance;
+import org.ektorp.DocumentNotFoundException;
 import org.ektorp.ViewQuery;
 import org.ektorp.http.HttpClient;
 import org.ektorp.http.StdHttpClient;
@@ -50,10 +63,7 @@ import org.ektorp.support.CouchDbDocument;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.util.*;
+import com.google.common.primitives.Ints;
 
 /**
  * Implementation of a CouchDB data store to be used by gora.
@@ -105,9 +115,10 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @param keyClass
    * @param persistentClass
    * @param properties
+   * @throws GoraException 
    */
   @Override
-  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) {
+  public void initialize(Class<K> keyClass, Class<T> persistentClass, Properties properties) throws GoraException {
     LOG.debug("Initializing CouchDB store");
     super.initialize(keyClass, persistentClass, properties);
 
@@ -131,10 +142,9 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
 
       db = new StdCouchDbConnector(mapping.getDatabaseName(), dbInstance, myObjectMapperFactory);
       db.createDatabaseIfNotExists();
-
-    } catch (IOException e) {
+    } catch (Exception e) {
       LOG.error("Error while initializing CouchDB store: {}", new Object[] { e.getMessage() });
-      throw new RuntimeException(e);
+      throw new GoraException(e);
     }
   }
 
@@ -164,29 +174,48 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * Create a new database in CouchDB if necessary.
    */
   @Override
-  public void createSchema() {
-    if (schemaExists()) {
-      return;
+  public void createSchema() throws GoraException {
+    try {
+      if (schemaExists()) {
+        return;
+      }
+      dbInstance.createDatabase(mapping.getDatabaseName());
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
-    dbInstance.createDatabase(mapping.getDatabaseName());
   }
 
   /**
    * Drop the database.
    */
   @Override
-  public void deleteSchema() {
-    if (schemaExists()) {
-      dbInstance.deleteDatabase(mapping.getDatabaseName());
-    }
+  public void deleteSchema() throws GoraException {
+    try {
+      if (schemaExists()) {
+        dbInstance.deleteDatabase(mapping.getDatabaseName());
+      }
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }     
   }
 
   /**
    * Check if the database already exists or should be created.
    */
   @Override
-  public boolean schemaExists() {
-    return dbInstance.checkIfDbExists(mapping.getDatabaseName());
+  public boolean schemaExists() throws GoraException {
+    try {
+      return dbInstance.checkIfDbExists(mapping.getDatabaseName());
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }
   }
 
   /**
@@ -196,15 +225,19 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @param fields list of fields to be loaded from the database
    */
   @Override
-  public T get(final K key, final String[] fields) {
+  public T get(final K key, final String[] fields) throws GoraException {
 
     final Map<String, Object>  result;
     try {
       result = db.get(Map.class, key.toString());
       return newInstance(result, getFieldsToQuery(fields));
+    } catch (DocumentNotFoundException e) {
+      return null ;
+    } catch (GoraException e) {
+      throw e;
     } catch (Exception e) {
-      LOG.info(e.getMessage(), e);
-      return null;
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -215,7 +248,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @param obj the object to be inserted
    */
   @Override
-  public void put(K key, T obj) {
+  public void put(K key, T obj) throws GoraException {
     final Map<String, Object> buffer = Collections.synchronizedMap(new LinkedHashMap<String, Object>());
     buffer.put("_id", key);
 
@@ -336,15 +369,20 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @return whether the object was successfully deleted
    */
   @Override
-  public boolean delete(K key) {
+  public boolean delete(K key) throws GoraException {
     if (key == null) {
       deleteSchema();
       createSchema();
       return true;
     }
-    final String keyString = key.toString();
-    final Map<String, Object> referenceData = db.get(Map.class, keyString);
-    return StringUtils.isNotEmpty(db.delete(keyString, referenceData.get("_rev").toString()));
+    try {
+      final String keyString = key.toString();
+      final Map<String, Object> referenceData = db.get(Map.class, keyString);
+      return StringUtils.isNotEmpty(db.delete(keyString, referenceData.get("_rev").toString()));
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }
   }
 
   /**
@@ -355,7 +393,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @return number of deleted records
    */
   @Override
-  public long deleteByQuery(Query<K, T> query) {
+  public long deleteByQuery(Query<K, T> query) throws GoraException {
 
     final K key = query.getKey();
     final K startKey = query.getStartKey();
@@ -366,24 +404,29 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
       createSchema();
       return -1;
     } else {
-      final ViewQuery viewQuery = new ViewQuery()
-          .allDocs()
-          .includeDocs(true)
-          .key(key)
-          .startKey(startKey)
-          .endKey(endKey);
-
-      final List<Map> result = db.queryView(viewQuery, Map.class);
-      final Map<String, List<String>> revisionsToPurge = new HashMap<>();
-
-      for (Map map : result) {
-        final List<String> revisions = new ArrayList<>();
-        String keyString = map.get("_id").toString();
-        String rev = map.get("_rev").toString();
-        revisions.add(rev);
-        revisionsToPurge.put(keyString, revisions);
+      try {
+        final ViewQuery viewQuery = new ViewQuery()
+            .allDocs()
+            .includeDocs(true)
+            .key(key)
+            .startKey(startKey)
+            .endKey(endKey);
+  
+        final List<Map> result = db.queryView(viewQuery, Map.class);
+        final Map<String, List<String>> revisionsToPurge = new HashMap<>();
+  
+        for (Map map : result) {
+          final List<String> revisions = new ArrayList<>();
+          String keyString = map.get("_id").toString();
+          String rev = map.get("_rev").toString();
+          revisions.add(rev);
+          revisionsToPurge.put(keyString, revisions);
+        }
+        return db.purge(revisionsToPurge).getPurged().size();
+      } catch (Exception e) {
+        LOG.error(e.getMessage(), e);
+        throw new GoraException(e);
       }
-      return db.purge(revisionsToPurge).getPurged().size();
     }
   }
 
@@ -401,18 +444,25 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * Execute the query and return the result.
    */
   @Override
-  public Result<K, T> execute(Query<K, T> query) {
-    query.setFields(getFieldsToQuery(query.getFields()));
-    final ViewQuery viewQuery = new ViewQuery()
-        .allDocs()
-        .includeDocs(true)
-        .startKey(query.getStartKey())
-        .endKey(query.getEndKey())
-        .limit(Ints.checkedCast(query.getLimit())); //FIXME GORA have long value but ektorp client use integer
-
-    CouchDBResult<K, T> couchDBResult = new CouchDBResult<>(this, query, db.queryView(viewQuery, Map.class));
-
-    return couchDBResult;
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
+
+    try {
+
+      query.setFields(getFieldsToQuery(query.getFields()));
+      final ViewQuery viewQuery = new ViewQuery()
+          .allDocs()
+          .includeDocs(true)
+          .startKey(query.getStartKey())
+          .endKey(query.getEndKey())
+          .limit(Ints.checkedCast(query.getLimit())); //FIXME GORA have long value but ektorp client use integer
+      CouchDBResult<K, T> couchDBResult = new CouchDBResult<>(this, query, db.queryView(viewQuery, Map.class));
+      return couchDBResult;
+
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e) ;
+    }
+
   }
 
   @Override
@@ -432,9 +482,9 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
    * @param result result from the query to the database
    * @param fields the list of fields to be mapped to the persistence class instance
    * @return a persistence class instance which content was deserialized
-   * @throws IOException
+   * @throws GoraException
    */
-  public T newInstance(Map<String, Object> result, String[] fields) throws IOException {
+  public T newInstance(Map<String, Object> result, String[] fields) throws GoraException {
     if (result == null)
       return null;
 
@@ -461,7 +511,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
 
   }
 
-  private Object fromCouchDBRecord(final Schema fieldSchema, final String docf, final Object value) {
+  private Object fromCouchDBRecord(final Schema fieldSchema, final String docf, final Object value) throws GoraException {
 
     final Object innerValue = ((Map) value).get(docf);
     if (innerValue == null) {
@@ -485,7 +535,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
     return record;
   }
 
-  private Object fromCouchDBMap(final Schema fieldSchema, final Field field, final String docf, final Object value) {
+  private Object fromCouchDBMap(final Schema fieldSchema, final Field field, final String docf, final Object value) throws GoraException {
 
     final Map<String, Object> map = (Map<String, Object>) ((Map<String, Object>) value).get(docf);
     final Map<Utf8, Object> rmap = new HashMap<>();
@@ -503,7 +553,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
     return new DirtyMapWrapper<>(rmap);
   }
 
-  private Object fromCouchDBUnion(final Schema fieldSchema, final Field field, final String docf, final Object value) {
+  private Object fromCouchDBUnion(final Schema fieldSchema, final Field field, final String docf, final Object value) throws GoraException {
 
     Object result;// schema [type0, type1]
     Schema.Type type0 = fieldSchema.getTypes().get(0).getType();
@@ -526,7 +576,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
     return result;
   }
 
-  private Object fromCouchDBList(final Schema fieldSchema, final Field field, final String docf, final Object value) {
+  private Object fromCouchDBList(final Schema fieldSchema, final Field field, final String docf, final Object value) throws GoraException {
     final List<Object> list = (List<Object>) ((Map<String, Object>) value).get(docf);
     final List<Object> rlist = new ArrayList<>();
 
@@ -574,7 +624,7 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
     return result;
   }
 
-  private Object fromDBObject(final Schema fieldSchema, final Field field, final String docf, final Object value) {
+  private Object fromDBObject(final Schema fieldSchema, final Field field, final String docf, final Object value) throws GoraException {
     if (value == null) {
       return null;
     }
@@ -615,14 +665,24 @@ public class CouchDBStore<K, T extends PersistentBase> extends DataStoreBase<K,
   }
 
   @Override
-  public void flush() {
-    db.executeBulk(bulkDocs);
-    bulkDocs.clear();
-    db.flushBulkBuffer();
+  public void flush() throws GoraException {
+    try {
+      db.executeBulk(bulkDocs);
+      bulkDocs.clear();
+      db.flushBulkBuffer();
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }
   }
 
   @Override
   public void close() {
-    flush();
+    try {
+      flush();
+    } catch (GoraException e) {
+      //Log and ignore. We are closing... so is doest not matter if it just died
+      LOG.warn("Error flushing when closing", e);
+    }
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-couchdb/src/test/java/org/apache/gora/couchdb/store/TestCouchDBStore.java
----------------------------------------------------------------------
diff --git a/gora-couchdb/src/test/java/org/apache/gora/couchdb/store/TestCouchDBStore.java b/gora-couchdb/src/test/java/org/apache/gora/couchdb/store/TestCouchDBStore.java
index b2cfc9a..ffcd0dd 100644
--- a/gora-couchdb/src/test/java/org/apache/gora/couchdb/store/TestCouchDBStore.java
+++ b/gora-couchdb/src/test/java/org/apache/gora/couchdb/store/TestCouchDBStore.java
@@ -25,6 +25,7 @@ import org.apache.gora.examples.generated.WebPage;
 import org.apache.gora.query.Query;
 import org.apache.gora.store.DataStore;
 import org.apache.gora.store.DataStoreTestBase;
+import org.apache.gora.util.GoraException;
 import org.junit.ClassRule;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -63,7 +64,7 @@ public class TestCouchDBStore extends DataStoreTestBase {
   }
 
   @Test
-  public void testPutAndGet() {
+  public void testPutAndGet() throws GoraException {
     WebPage page = webPageStore.newPersistent();
 
     // Write webpage data

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/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
index bebd06c..56e673f 100644
--- 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
@@ -29,6 +29,7 @@ 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;
+import org.apache.gora.util.GoraException;
 
 public class DynamoDBAvroStore<K, T extends PersistentBase> extends
 DataStoreBase<K, T> implements IDynamoDB<K, T> {
@@ -62,7 +63,7 @@ DataStoreBase<K, T> implements IDynamoDB<K, T> {
   }
 
   @Override
-  public void createSchema() {
+  public void createSchema() throws GoraException {
     // TODO Auto-generated method stub
 
   }
@@ -80,13 +81,13 @@ DataStoreBase<K, T> implements IDynamoDB<K, T> {
   }
 
   @Override
-  public void deleteSchema() {
+  public void deleteSchema() throws GoraException {
     // TODO Auto-generated method stub
 
   }
 
   @Override
-  public Result<K, T> execute(Query<K, T> arg0) {
+  public Result<K, T> execute(Query<K, T> arg0) throws GoraException {
     // TODO Auto-generated method stub
     return null;
   }
@@ -98,7 +99,7 @@ DataStoreBase<K, T> implements IDynamoDB<K, T> {
   }
 
   @Override
-  public T get(K arg0, String[] arg1) {
+  public T get(K arg0, String[] arg1) throws GoraException {
     // TODO Auto-generated method stub
     return null;
   }
@@ -128,7 +129,7 @@ DataStoreBase<K, T> implements IDynamoDB<K, T> {
   }
 
   @Override
-  public boolean schemaExists() {
+  public boolean schemaExists() throws GoraException {
     // TODO Auto-generated method stub
     return false;
   }

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/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
index 24beda5..b2b0da2 100644
--- 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
@@ -65,16 +65,17 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    */
   @Override
   @SuppressWarnings("unchecked")
-  public long deleteByQuery(Query<K, T> query) {
+  public long deleteByQuery(Query<K, T> query) throws GoraException {
     // 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>();
+    ArrayList<T> deletes = null ;
     try {
+      Result<K, T> result = execute(query);
+      deletes = new ArrayList<T>();
       while (result.next()) {
         T resultObj = result.get();
         deletes.add(resultObj);
@@ -87,18 +88,11 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
         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 (GoraException e) {
+      throw e ; // If it is a GoraException we assume it is already logged
     } catch (Exception e) {
-      LOG.error(e.getMessage());
-      throw new RuntimeException(e);
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
     return deletes.size();
   }
@@ -108,22 +102,27 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * 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);
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
+    try {
+      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);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }
   }
 
   @Override
-  public T get(K key, String[] fields) {
+  public T get(K key, String[] fields) throws GoraException {
     /*
      * DynamoDBQuery<K,T> query = new DynamoDBQuery<K,T>();
      * query.setDataStore(this); //query.setKeyRange(key, key);
@@ -139,7 +138,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * Gets the object with the specific key
    * @throws IOException
    */
-  public T get(K key) {
+  public T get(K key) throws GoraException {
     T object = null;
     try {
       Object rangeKey;
@@ -152,23 +151,18 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
           object = mapper.load(persistentClass, hashKey, rangeKey);
         else
           object = mapper.load(persistentClass, hashKey);
-      } else
+        return object;
+        
+      } 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());
+      }
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
-    return object;
   }
 
   /**
@@ -186,7 +180,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * @return
    */
   @Override
-  public K newKey() {
+  public K newKey() throws GoraException {
     // TODO Auto-generated method stub
     return null;
   }
@@ -197,16 +191,16 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * @return
    */
   @Override
-  public T newPersistent() {
+  public T newPersistent() throws GoraException {
     T obj = null;
     try {
       obj = persistentClass.newInstance();
     } catch (InstantiationException e) {
-      LOG.error("Error instantiating " + persistentClass.getCanonicalName());
-      throw new InstantiationError(e.getMessage());
+      LOG.error("Error instantiating " + persistentClass.getCanonicalName(), e);
+      throw new GoraException(e);
     } catch (IllegalAccessException e) {
-      LOG.error("Error instantiating " + persistentClass.getCanonicalName());
-      throw new IllegalAccessError(e.getMessage());
+      LOG.error("Error instantiating " + persistentClass.getCanonicalName(),e );
+      throw new GoraException(e);
     }
     return obj;
   }
@@ -218,7 +212,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * @param obj
    */
   @Override
-  public void put(K key, T obj) {
+  public void put(K key, T obj) throws GoraException {
     try {
       Object hashKey = getHashKey(key, obj);
       Object rangeKey = getRangeKey(key, obj);
@@ -233,12 +227,9 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
         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);
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -249,7 +240,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * @return true for a successful process
    */
   @Override
-  public boolean delete(K key) {
+  public boolean delete(K key) throws GoraException {
     try {
       T object = null;
       Object rangeKey = null, hashKey = null;
@@ -283,9 +274,8 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
       mapper.delete(object);
       return true;
     } catch (Exception e) {
-      LOG.error("Error while deleting value with key " + key.toString());
-      LOG.error(e.getMessage());
-      return false;
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -295,7 +285,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
    * reading the mapping file
    */
   public void initialize(Class<K> keyClass, Class<T> pPersistentClass,
-      Properties properties) {
+      Properties properties) throws GoraException {
     super.initialize(keyClass, pPersistentClass, properties);
     setWsProvider(WS_PROVIDER);
     if (autoCreateSchema) {
@@ -334,7 +324,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
 
   @Override
   public void flush() {
-    LOG.warn("DynamoDBNativeStore puts and gets directly into the datastore");
+    LOG.info("DynamoDBNativeStore puts and gets directly into the datastore");
   }
 
   @Override
@@ -357,30 +347,35 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
   }
 
   @Override
-  public void createSchema() {
+  public void createSchema() throws GoraException {
     LOG.info("Creating Native DynamoDB Schemas.");
     if (dynamoDBStoreHandler.getDynamoDbMapping().getTables().isEmpty()) {
-      throw new IllegalStateException("There are not tables defined.");
+      throw new GoraException("There are not tables defined.");
     }
-    if (dynamoDBStoreHandler.getPreferredSchema() == null) {
-      LOG.debug("Creating schemas.");
-      // read the mapping object
-      for (String tableName : dynamoDBStoreHandler.getDynamoDbMapping()
-          .getTables().keySet())
+    try {
+      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));
-      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));
+      }
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -397,7 +392,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
   }
 
   @Override
-  public void deleteSchema() {
+  public void deleteSchema() throws GoraException {
     // TODO Auto-generated method stub
 
   }
@@ -408,7 +403,7 @@ public class DynamoDBNativeStore<K, T extends Persistent> extends
   }
 
   @Override
-  public boolean schemaExists() {
+  public boolean schemaExists() throws GoraException {
     return this.dynamoDBStoreHandler.schemaExists();
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/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 da190b4..f5e53a3 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
@@ -107,55 +107,60 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
    * group of schemas defined within the mapping file
    */
   @Override
-  public void createSchema() {
+  public void createSchema() throws GoraException {
     dynamoDbStore.createSchema();
   }
 
   @Override
-  public boolean delete(K key) {
+  public boolean delete(K key) throws GoraException {
     return dynamoDbStore.delete(key);
   }
 
   @Override
-  public long deleteByQuery(Query<K, T> query) {
+  public long deleteByQuery(Query<K, T> query) throws GoraException {
     return dynamoDbStore.deleteByQuery(query);
   }
 
   @Override
-  public void deleteSchema() {
-    if (getDynamoDbMapping().getTables().isEmpty())
-      throw new IllegalStateException("There are not tables defined.");
-    if (preferredSchema == null) {
-      LOG.debug("Delete schemas");
+  public void deleteSchema() throws GoraException {
+    try {
       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);
+        return ; // Nothing to delete
+      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 (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public Result<K, T> execute(Query<K, T> query) {
+  public Result<K, T> execute(Query<K, T> query) throws GoraException {
     return dynamoDbStore.execute(query);
   }
 
   @Override
-  public void flush() {
+  public void flush() throws GoraException {
     dynamoDbStore.flush();
   }
 
   @Override
-  public T get(K key) {
+  public T get(K key) throws GoraException {
     return dynamoDbStore.get(key);
   }
 
   @Override
-  public T get(K key, String[] fields) {
+  public T get(K key, String[] fields) throws GoraException {
     return dynamoDbStore.get(key, fields);
   }
 
@@ -200,7 +205,7 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
    */
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-      Properties properties) {
+      Properties properties) throws GoraException {
     try {
       LOG.debug("Initializing DynamoDB store");
       setDynamoDBProperties(properties);
@@ -208,9 +213,11 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
       dynamoDbStore = DynamoDBFactory.buildDynamoDBStore(getSerializationType());
       dynamoDbStore.setDynamoDBStoreHandler(this);
       dynamoDbStore.initialize(keyClass, persistentClass, properties);
+    } catch (GoraException e) {
+      throw e;
     } catch (Exception e) {
-      LOG.error("Error while initializing DynamoDB store", e.getMessage());
-      throw new RuntimeException(e);
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -226,12 +233,12 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
   }
 
   @Override
-  public K newKey() {
+  public K newKey() throws GoraException {
     return dynamoDbStore.newKey();
   }
 
   @Override
-  public T newPersistent() {
+  public T newPersistent() throws GoraException {
     return dynamoDbStore.newPersistent();
   }
 
@@ -241,7 +248,7 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
   }
 
   @Override
-  public void put(K key, T value) {
+  public void put(K key, T value) throws GoraException {
     dynamoDbStore.put(key, value);
   }
 
@@ -251,27 +258,32 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
    * @return
    */
   @Override
-  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");
+  public boolean schemaExists() throws GoraException {
+    try {
+      LOG.info("Verifying schemas.");
+      TableDescription success = null;
       if (getDynamoDbMapping().getTables().isEmpty())
         throw new IllegalStateException("There are not tables defined.");
-      // read the mapping object
-      for (String tableName : getDynamoDbMapping().getTables().keySet()) {
-        success = getTableSchema(tableName);
-        if (success == null)
-          return false;
+      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 : getDynamoDbMapping().getTables().keySet()) {
+          success = getTableSchema(tableName);
+          if (success == null)
+            return false;
+        }
+      } else {
+        LOG.info("Verifying schema " + preferredSchema);
+        success = getTableSchema(preferredSchema);
       }
-    } else {
-      LOG.info("Verifying schema " + preferredSchema);
-      success = getTableSchema(preferredSchema);
+      LOG.info("Finished verifying schemas.");
+      return (success != null) ? true : false;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
-    LOG.info("Finished verifying schemas.");
-    return (success != null) ? true : false;
   }
 
   @Override
@@ -290,7 +302,7 @@ public class DynamoDBStore<K, T extends Persistent> implements DataStore<K, T> {
   }
 
   @Override
-  public void truncateSchema() {
+  public void truncateSchema() throws GoraException {
     // TODO Auto-generated method stub
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
----------------------------------------------------------------------
diff --git a/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java b/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
index fe546aa..349c6d9 100644
--- a/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
+++ b/gora-hbase/src/main/java/org/apache/gora/hbase/store/HBaseStore.java
@@ -34,6 +34,8 @@ import java.util.NavigableMap;
 import java.util.Properties;
 import java.util.Set;
 
+import javax.naming.ConfigurationException;
+
 import org.apache.avro.Schema;
 import org.apache.avro.Schema.Field;
 import org.apache.avro.Schema.Type;
@@ -52,6 +54,7 @@ import org.apache.gora.query.Query;
 import org.apache.gora.query.impl.PartitionQueryImpl;
 import org.apache.gora.store.DataStoreFactory;
 import org.apache.gora.store.impl.DataStoreBase;
+import org.apache.gora.util.GoraException;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseConfiguration;
@@ -73,8 +76,6 @@ import org.jdom.input.SAXBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import javax.naming.ConfigurationException;
-
 /**
  * DataStore for HBase. Thread safe.
  *
@@ -123,19 +124,20 @@ implements Configurable {
    */
   @Override
   public void initialize(Class<K> keyClass, Class<T> persistentClass,
-      Properties properties) {
-    try {
+      Properties properties) throws GoraException {
       super.initialize(keyClass, persistentClass, properties);
 
+    try {
       this.conf = HBaseConfiguration.create(getConf());
       admin = ConnectionFactory.createConnection(getConf()).getAdmin();
       mapping = readMapping(getConf().get(PARSE_MAPPING_FILE_KEY, DEFAULT_MAPPING_FILE));
       filterUtil = new HBaseFilterUtil<>(this.conf);
     } catch (FileNotFoundException ex) {
       LOG.error("{}  is not found, please check the file.", DEFAULT_MAPPING_FILE);
-      throw new RuntimeException(ex);
+      throw new GoraException(ex);
     } catch (Exception e) {
-      throw new RuntimeException(e);
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
 
     // Set scanner caching option
@@ -144,8 +146,8 @@ implements Configurable {
           Integer.valueOf(DataStoreFactory.findProperty(this.properties, this,
               SCANNER_CACHING_PROPERTIES_KEY,
               String.valueOf(SCANNER_CACHING_PROPERTIES_DEFAULT)))) ;
-    }catch(Exception e){
-      LOG.error("Can not load {} from gora.properties. Setting to default value: {}.", SCANNER_CACHING_PROPERTIES_KEY, SCANNER_CACHING_PROPERTIES_DEFAULT);
+    }catch(NumberFormatException e){
+      LOG.info("Can not load {} from gora.properties. Setting to default value: {}.", SCANNER_CACHING_PROPERTIES_KEY, SCANNER_CACHING_PROPERTIES_DEFAULT);
       this.setScannerCaching(SCANNER_CACHING_PROPERTIES_DEFAULT) ; // Default value if something is wrong
     }
 
@@ -155,8 +157,9 @@ implements Configurable {
     try{
       boolean autoflush = this.conf.getBoolean("hbase.client.autoflush.default", false);
       table = new HBaseTableConnection(getConf(), getSchemaName(), autoflush);
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
     closeHBaseAdmin();
   }
@@ -172,7 +175,7 @@ implements Configurable {
   }
 
   @Override
-  public void createSchema() {
+  public void createSchema() throws GoraException {
     try{
       if(schemaExists()) {
         return;
@@ -180,47 +183,55 @@ implements Configurable {
       HTableDescriptor tableDesc = mapping.getTable();
   
       admin.createTable(tableDesc);
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
     closeHBaseAdmin();
   }
 
   @Override
-  public void deleteSchema() {
+  public void deleteSchema() throws GoraException {
     try{
       if(!schemaExists()) {
         return;
       }
       admin.disableTable(mapping.getTable().getTableName());
       admin.deleteTable(mapping.getTable().getTableName());
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
     closeHBaseAdmin();
   }
 
   @Override
-  public boolean schemaExists() {
+  public boolean schemaExists() throws GoraException {
     try{
       return admin.tableExists(mapping.getTable().getTableName());
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
-      return false;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public T get(K key, String[] fields) {
+  public T get(K key, String[] fields) throws GoraException {
     try{
       fields = getFieldsToQuery(fields);
       Get get = new Get(toBytes(key));
       addFields(get, fields);
       Result result = table.get(get);
       return newInstance(result, fields);
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
-      return null;
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -237,7 +248,7 @@ implements Configurable {
    *          Record to be persisted in HBase
    */
   @Override
-  public void put(K key, T persistent) {
+  public void put(K key, T persistent) throws GoraException {
     try {
       Schema schema = persistent.getSchema();
       byte[] keyRaw = toBytes(key);
@@ -255,9 +266,11 @@ implements Configurable {
         Object o = persistent.get(i);
         HBaseColumn hcol = mapping.getColumn(field.name());
         if (hcol == null) {
-          throw new RuntimeException("HBase mapping for field ["
+          String errorMsg = "HBase mapping for field ["
               + persistent.getClass().getName() + "#" + field.name()
-              + "] not found. Wrong gora-hbase-mapping.xml?");
+              + "] not found. Wrong gora-hbase-mapping.xml?";
+          LOG.error(errorMsg);
+          throw new GoraException(errorMsg);
         }
         addPutsAndDeletes(put, delete, o, field.schema().getType(),
             field.schema(), hcol, hcol.getQualifier());
@@ -265,14 +278,15 @@ implements Configurable {
 
       if (delete.size() > 0) {
         table.delete(delete);
-//        table.delete(delete);
-//        table.delete(delete); // HBase sometimes does not delete arbitrarily
       }
       if (put.size() > 0) {
         table.put(put);
       }
-    } catch (IOException ex2) {
-      LOG.error(ex2.getMessage(), ex2);
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -282,10 +296,8 @@ implements Configurable {
     case UNION:
       if (isNullable(schema) && o == null) {
         if (qualifier == null) {
-//          delete.deleteFamily(hcol.getFamily());
           delete.addFamily(hcol.getFamily());
         } else {
-//          delete.deleteColumn(hcol.getFamily(), qualifier);
           delete.addColumns(hcol.getFamily(), qualifier);
         }
       } else {
@@ -305,10 +317,8 @@ implements Configurable {
       // if it's a map that has been modified, then the content should be replaced by the new one
       // This is because we don't know if the content has changed or not.
       if (qualifier == null) {
-        //delete.deleteFamily(hcol.getFamily());
         delete.addFamily(hcol.getFamily());
       } else {
-        //delete.deleteColumn(hcol.getFamily(), qualifier);
         delete.addColumns(hcol.getFamily(), qualifier);
       }
       @SuppressWarnings({ "rawtypes", "unchecked" })
@@ -352,20 +362,20 @@ implements Configurable {
    * @return always true
    */
   @Override
-  public boolean delete(K key) {
+  public boolean delete(K key) throws GoraException {
     try{
       table.delete(new Delete(toBytes(key)));
       //HBase does not return success information and executing a get for
       //success is a bit costly
       return true;
-    } catch(IOException ex2){
-      LOG.error(ex2.getMessage(), ex2);
-      return false;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public long deleteByQuery(Query<K, T> query) {
+  public long deleteByQuery(Query<K, T> query) throws GoraException {
     try {
       String[] fields = getFieldsToQuery(query.getFields());
       //find whether all fields are queried, which means that complete
@@ -384,18 +394,21 @@ implements Configurable {
       }
       table.delete(deletes);
       return deletes.size();
-    } catch (Exception ex) {
-      LOG.error(ex.getMessage(), ex);
-      return -1;
+    } catch (GoraException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
   @Override
-  public void flush() {
+  public void flush() throws GoraException {
     try{
       table.flushCommits();
-    }catch(IOException ex){
-      LOG.error(ex.getMessage(), ex);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
@@ -457,7 +470,7 @@ implements Configurable {
   }
 
   @Override
-  public org.apache.gora.query.Result<K, T> execute(Query<K, T> query){
+  public org.apache.gora.query.Result<K, T> execute(Query<K, T> query) throws GoraException {
     try{
       //check if query.fields is null
       query.setFields(getFieldsToQuery(query.getFields()));
@@ -479,7 +492,7 @@ implements Configurable {
       }
     }catch(IOException ex){
       LOG.error(ex.getMessage(), ex);
-      return null;
+      throw new GoraException(ex) ;
     }
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/b06da5f3/gora-infinispan/src/main/java/org/apache/gora/infinispan/store/InfinispanClient.java
----------------------------------------------------------------------
diff --git a/gora-infinispan/src/main/java/org/apache/gora/infinispan/store/InfinispanClient.java b/gora-infinispan/src/main/java/org/apache/gora/infinispan/store/InfinispanClient.java
index 335c85b..a41245c 100644
--- a/gora-infinispan/src/main/java/org/apache/gora/infinispan/store/InfinispanClient.java
+++ b/gora-infinispan/src/main/java/org/apache/gora/infinispan/store/InfinispanClient.java
@@ -17,7 +17,12 @@
  */
 package org.apache.gora.infinispan.store;
 
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
 import org.apache.gora.persistency.impl.PersistentBase;
+import org.apache.gora.util.GoraException;
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.conf.Configuration;
 import org.infinispan.avro.client.Marshaller;
@@ -31,10 +36,6 @@ import org.infinispan.commons.api.BasicCache;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
 /*
  * @author Pierre Sutra, Valerio Schiavoni
  */
@@ -93,22 +94,28 @@ public class InfinispanClient<K, T extends PersistentBase> implements Configurab
     return cacheExists;
   }
 
-  public synchronized void createSchema() {
+  public synchronized void createSchema() throws GoraException {
     try {
       Support.registerSchema(cacheManager, persistentClass.newInstance().getSchema());
-    } catch (InstantiationException | IllegalAccessException e) {
-      throw new RuntimeException(e);
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
     }
   }
 
-  public void createCache() {
+  public void createCache() throws GoraException {
     createSchema();
     cacheExists = true;
   }
 
-  public void dropCache() {
-    cache.clear();
-    cacheExists = false;
+  public void dropCache() throws GoraException {
+    try {
+      cache.clear();
+      cacheExists = false;
+    } catch (Exception e) {
+      LOG.error(e.getMessage(), e);
+      throw new GoraException(e);
+    }
   }
 
   public void deleteByKey(K key) {