You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@gora.apache.org by dj...@apache.org on 2018/10/08 09:02:46 UTC

[11/11] gora git commit: Improve SQL building strategy and add MR tests for Ignite

Improve SQL building strategy and add MR tests for Ignite

The IgniteSQLBuilder class was refactored in in order to simplify the
code. Also, an implementation of DataStoreMapReduceTestBase was added
to the test packages. Finally, some missing formatting issues were
solved.

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

Branch: refs/heads/master
Commit: 446f42ea3685691ee10cf5054a055e8f1685077f
Parents: 8220a15
Author: Carlos M <ca...@gmail.com>
Authored: Sun Sep 9 23:30:53 2018 -0500
Committer: Carlos M <ca...@gmail.com>
Committed: Sun Sep 9 23:30:53 2018 -0500

----------------------------------------------------------------------
 .gitignore                                      |   1 -
 gora-ignite/pom.xml                             |   9 +-
 .../apache/gora/ignite/query/IgniteQuery.java   |   5 +
 .../apache/gora/ignite/query/IgniteResult.java  |  19 +-
 .../apache/gora/ignite/query/package-info.java  |   4 +-
 .../org/apache/gora/ignite/store/Column.java    |  32 ++-
 .../apache/gora/ignite/store/IgniteMapping.java |  33 +++
 .../gora/ignite/store/IgniteMappingBuilder.java |  22 +-
 .../gora/ignite/store/IgniteParameters.java     | 111 ++++++----
 .../apache/gora/ignite/store/IgniteStore.java   |  67 +++---
 .../apache/gora/ignite/store/package-info.java  |   4 +-
 .../ignite/utils/IgniteBackendConstants.java    |  41 +++-
 .../gora/ignite/utils/IgniteSQLBuilder.java     | 209 ++++++++++---------
 .../apache/gora/ignite/utils/package-info.java  |   2 +-
 .../mapreduce/IgniteStoreMapReduceTest.java     |  65 ++++++
 .../gora/ignite/mapreduce/package-info.java     |  20 ++
 .../org/apache/gora/ignite/package-info.java    |  20 ++
 .../apache/gora/ignite/store/package-info.java  |  20 ++
 pom.xml                                         | 117 ++++++-----
 19 files changed, 546 insertions(+), 255 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index d4f78bc..2aa54cc 100644
--- a/.gitignore
+++ b/.gitignore
@@ -37,4 +37,3 @@ ivy/ivy*.jar
 **/gora-solr-5/src/test/conf/solr/collection1/data/
 **/_rest_managed.json
 **/gora-solr/src/test/conf/solr-managed-schema/collection1/data/
-**/nbactions.xml

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/pom.xml
----------------------------------------------------------------------
diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml
index 8f75553..3e68e3b 100644
--- a/gora-ignite/pom.xml
+++ b/gora-ignite/pom.xml
@@ -110,7 +110,6 @@
     <dependency>
       <groupId>org.apache.ignite</groupId>
       <artifactId>ignite-core</artifactId>
-      <version>${ignite.version}</version>
     </dependency>
 
     <dependency>
@@ -165,8 +164,12 @@
     <dependency>
       <groupId>org.apache.ignite</groupId>
       <artifactId>ignite-indexing</artifactId>
-      <version>${ignite.version}</version>
     </dependency>
-  </dependencies>
 
+    <dependency>
+      <groupId>com.healthmarketscience.sqlbuilder</groupId>
+      <artifactId>sqlbuilder</artifactId>
+    </dependency>
+  </dependencies>
+  
 </project>

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java
index 7ad9c61..21715db 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java
@@ -19,6 +19,7 @@ package org.apache.gora.ignite.query;
 import org.apache.gora.persistency.impl.PersistentBase;
 import org.apache.gora.query.impl.QueryBase;
 import org.apache.gora.store.DataStore;
+import org.apache.gora.query.Query;
 
 /**
  * Ignite specific implementation of the {@link Query} interface.
@@ -28,5 +29,9 @@ public class IgniteQuery<K, T extends PersistentBase> extends QueryBase<K, T> {
   public IgniteQuery(DataStore<K, T> dataStore) {
     super(dataStore);
   }
+  
+  public IgniteQuery() {
+    super(null);
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java
index e5c0d46..913e688 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java
@@ -35,8 +35,15 @@ public class IgniteResult<K, T extends PersistentBase> extends ResultBase<K, T>
   private ResultSet resultSet;
   private int size;
 
-  public IgniteResult(DataStore<K, T> dataStore, Query<K, T> query) {
+  public IgniteResult(DataStore<K, T> dataStore, Query<K, T> query, ResultSet resultSet) throws SQLException {
     super(dataStore, query);
+    this.resultSet = resultSet;
+    if (resultSet.last()) {
+      size = resultSet.getRow();
+    } else {
+      size = 0;
+    }
+    resultSet.beforeFirst();
   }
 
   @Override
@@ -80,14 +87,4 @@ public class IgniteResult<K, T extends PersistentBase> extends ResultBase<K, T>
     }
   }
 
-  public void setResultSet(ResultSet resultSet) throws SQLException {
-    this.resultSet = resultSet;
-    if (resultSet.last()) {
-      size = resultSet.getRow();
-    } else {
-      size = 0;
-    }
-    resultSet.beforeFirst();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java
index fa52d9c..0cbd9af 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java
@@ -15,6 +15,6 @@
  * limitations under the License.
  */
 /**
- * This package contains Ignite datastore related all classes.
+ * This package contains Query related classes.
  */
-package org.apache.gora.ignite.query;
\ No newline at end of file
+package org.apache.gora.ignite.query;

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java
index ee7d6d6..3034cdf 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java
@@ -21,32 +21,56 @@ public class Column {
   private String name;
   private FieldType dataType;
 
+  /**
+   * Constructor for Column
+   *
+   * @param name Column's name
+   * @param dataType Column's data type
+   */
   public Column(String name, FieldType dataType) {
     this.name = name;
     this.dataType = dataType;
   }
 
+  /**
+   * Returns the column's name
+   *
+   * @return Column's name
+   */
   public String getName() {
     return name;
   }
 
+  /**
+   * Sets the column's name
+   *
+   * @param name Column's name
+   */
   public void setName(String name) {
     this.name = name;
   }
 
+  /**
+   * Returns the column's data-type
+   *
+   * @return Column's data-type
+   */
   public FieldType getDataType() {
     return dataType;
   }
 
+  /**
+   * Sets the column's data-type
+   *
+   * @param dataType Column's data-type
+   */
   public void setDataType(FieldType dataType) {
     this.dataType = dataType;
   }
-  
-  
 
   /**
-   * For a more detailed list of data types supported by Ignite and its
-   * equivalents in Java refer to
+   * Ignite supported data-type enumeration. For a more detailed list of data
+   * types supported by Ignite and its equivalents in Java refer to
    * https://apacheignite-sql.readme.io/docs/data-types
    */
   public static enum FieldType {

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java
index f0189ec..4540ad4 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java
@@ -29,30 +29,63 @@ public class IgniteMapping {
   private Map<String, Column> fields;
   private List<Column> primaryKey;
 
+  /**
+   * Empty constructor for the IgniteMapping class
+   */
   public IgniteMapping() {
     fields = new HashMap<>();
   }
 
+  /**
+   * Returns the name of ignite table linked to the mapping.
+   *
+   * @return Table's name.
+   */
   public String getTableName() {
     return tableName;
   }
 
+  /**
+   * Sets the table name of the ignite mapping
+   *
+   * @param tableName Table's name
+   */
   public void setTableName(String tableName) {
     this.tableName = tableName;
   }
 
+  /**
+   * Returns a map with all field-column mappings
+   *
+   * @return Map containing mapped fields
+   */
   public Map<String, Column> getFields() {
     return fields;
   }
 
+  /**
+   * Sets field-column mappings
+   *
+   * @param fields Map containing mapped fields
+   */
   public void setFields(Map<String, Column> fields) {
     this.fields = fields;
   }
 
+  /**
+   * Returns the primary key's list of columns
+   *
+   * @return List with columns
+   */
   public List<Column> getPrimaryKey() {
     return primaryKey;
   }
 
+  /**
+   * Sets the primary key's columns
+   *
+   * @param primaryKey List with columns
+   */
   public void setPrimaryKey(List<Column> primaryKey) {
     this.primaryKey = primaryKey;
   }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java
index 4d397c5..83ad3ba 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java
@@ -44,19 +44,39 @@ public class IgniteMappingBuilder<K, T extends PersistentBase> {
 
   private final IgniteStore<K, T> dataStore;
 
+  /**
+   * Constructor for IgniteMappingBuilder
+   *
+   * @param store IgniteStore instance
+   */
   public IgniteMappingBuilder(final IgniteStore<K, T> store) {
     this.igniteMapping = new IgniteMapping();
     this.dataStore = store;
   }
 
+  /**
+   * Returns the Ignite Mapping being built
+   *
+   * @return Ignite Mapping instance
+   */
   public IgniteMapping getIgniteMapping() {
     return igniteMapping;
   }
 
+  /**
+   * Sets the Ignite Mapping
+   *
+   * @param igniteMapping Ignite Mapping instance
+   */
   public void setIgniteMapping(IgniteMapping igniteMapping) {
     this.igniteMapping = igniteMapping;
   }
 
+  /**
+   * Reads Ignite mappings from file
+   *
+   * @param mappingFile File name relative to the resource's classpath
+   */
   public void readMappingFile(String mappingFile) {
     try {
       SAXBuilder saxBuilder = new SAXBuilder();
@@ -102,11 +122,9 @@ public class IgniteMappingBuilder<K, T extends PersistentBase> {
           break;
         }
       }
-
     } catch (IOException | JDOMException | ConfigurationException e) {
       throw new RuntimeException(e);
     }
     LOG.info("Gora Ignite mapping file was read successfully.");
-
   }
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java
index f667b40..6606583 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java
@@ -18,43 +18,12 @@ package org.apache.gora.ignite.store;
 
 import java.util.Properties;
 import org.apache.gora.ignite.utils.IgniteBackendConstants;
-import org.apache.hadoop.conf.Configuration;
 
 /**
  * Parameters definitions for Ignite.
  */
 public class IgniteParameters {
 
-  /**
-   * Property indicating the Ignite Schema to be used
-   */
-  public static final String PROP_SCHEMA = "gora.datastore.ignite.schema";
-
-  /**
-   * Property indicating the Ignite Cluster Node IP
-   */
-  public static final String PROP_HOST = "gora.datastore.ignite.host";
-
-  /**
-   * Property indicating the port used by the Ignite Server
-   */
-  public static final String PROP_PORT = "gora.datastore.ignite.port";
-
-  /**
-   * Property indicating the username to connect to the server
-   */
-  public static final String PROP_USER = "gora.datastore.ignite.user";
-
-  /**
-   * Property indicating the password to connect to the server
-   */
-  public static final String PROP_PASSWORD = "gora.datastore.ignite.password";
-
-  /**
-   * Property indicating additional JDBC options
-   */
-  public static final String PROP_ADDITIONALS = "gora.datastore.ignite.additionalConfigurations";
-
   private String host;
   private String port;
   private String schema;
@@ -79,62 +48,130 @@ public class IgniteParameters {
     this.additionalConfigurations = additionalConfigurations;
   }
 
+  /**
+   * Returns the ignite hostname
+   *
+   * @return IP/domain of the ignite server
+   */
   public String getHost() {
     return host;
   }
 
+  /**
+   * Sets the ignite hostname
+   *
+   * @param host IP/domain of the ignite server
+   */
   public void setHost(String host) {
     this.host = host;
   }
 
+  /**
+   * Returns the ignite port
+   *
+   * @return Port of the ignite server eg. 10800
+   */
   public String getPort() {
     return port;
   }
 
+  /**
+   * Sets the ignite port
+   *
+   * @param port Port of the ignite server eg. 10800
+   */
   public void setPort(String port) {
     this.port = port;
   }
 
+  /**
+   * Returns the username used for the ignite connection
+   *
+   * @return Username of ignite
+   */
   public String getUser() {
     return user;
   }
 
+  /**
+   * Sets the username used for the ignite connection
+   *
+   * @param user Username of ignite
+   */
   public void setUser(String user) {
     this.user = user;
   }
 
+  /**
+   * Returns the secrets used for the ignite connection
+   *
+   * @return Password of the ignite user
+   */
   public String getPassword() {
     return password;
   }
 
+  /**
+   * Sets the secrets used for the ignite connection
+   *
+   * @param password Password of the ignite user
+   */
   public void setPassword(String password) {
     this.password = password;
   }
 
+  /**
+   * Returns additional configurations used for the JDBC connection For more
+   * details refer to https://apacheignite-sql.readme.io/docs/jdbc-driver
+   *
+   * @return String containing JDBC configurations
+   */
   public String getAdditionalConfigurations() {
     return additionalConfigurations;
   }
 
+  /**
+   * Sets additional configurations used for the JDBC connection. For more
+   * details refer to https://apacheignite-sql.readme.io/docs/jdbc-driver
+   *
+   * @param additionalConfigurations String containing JDBC configurations
+   */
   public void setAdditionalConfigurations(String additionalConfigurations) {
     this.additionalConfigurations = additionalConfigurations;
   }
 
+  /**
+   * Returns the ignite schema for the JDBC connection
+   *
+   * @return Ignite schema e.g. PUBLIC
+   */
   public String getSchema() {
     return schema;
   }
 
+  /**
+   * Sets the ignite schema for the JDBC connection
+   *
+   * @param schema Ignite schema e.g. PUBLIC
+   */
   public void setSchema(String schema) {
     this.schema = schema;
   }
 
+  /**
+   * Reads Ignite parameters from a properties list
+   *
+   * @param properties Properties list
+   * @return Ignite parameters instance
+   */
   public static IgniteParameters load(Properties properties) {
     return new IgniteParameters(
-        properties.getProperty(PROP_HOST, IgniteBackendConstants.DEFAULT_IGNITE_HOST),
-        properties.getProperty(PROP_PORT, IgniteBackendConstants.DEFAULT_IGNITE_PORT),
-        properties.getProperty(PROP_SCHEMA, null),
-        properties.getProperty(PROP_USER, null),
-        properties.getProperty(PROP_PASSWORD, null),
-        properties.getProperty(PROP_ADDITIONALS, null));
+        properties.getProperty(IgniteBackendConstants.PROP_HOST, IgniteBackendConstants.DEFAULT_IGNITE_HOST),
+        properties.getProperty(IgniteBackendConstants.PROP_PORT, IgniteBackendConstants.DEFAULT_IGNITE_PORT),
+        properties.getProperty(IgniteBackendConstants.PROP_SCHEMA),
+        properties.getProperty(IgniteBackendConstants.PROP_USER),
+        properties.getProperty(IgniteBackendConstants.PROP_PASSWORD),
+        properties.getProperty(IgniteBackendConstants.PROP_ADDITIONALS));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java
index a17b3a0..4f1abc9 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java
@@ -81,6 +81,9 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
       igniteParameters = IgniteParameters.load(properties);
       connection = acquireConnection();
       LOG.info("Ignite store was successfully initialized");
+      if (!schemaExists()) {
+        createSchema();
+      }
     } catch (ClassNotFoundException | SQLException ex) {
       LOG.error("Error while initializing Ignite store", ex);
       throw new GoraException(ex);
@@ -88,9 +91,9 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
   }
 
   private Connection acquireConnection() throws ClassNotFoundException, SQLException {
-    Class.forName("org.apache.ignite.IgniteJdbcThinDriver");
+    Class.forName(IgniteBackendConstants.DRIVER_NAME);
     StringBuilder urlBuilder = new StringBuilder();
-    urlBuilder.append("jdbc:ignite:thin://");
+    urlBuilder.append(IgniteBackendConstants.JDBC_PREFIX);
     urlBuilder.append(igniteParameters.getHost());
     if (igniteParameters.getPort() != null) {
       urlBuilder.append(":" + igniteParameters.getPort());
@@ -187,9 +190,9 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
     for (String af : avFields) {
       dbFields.add(igniteMapping.getFields().get(af).getName());
     }
-    String selectQuery = IgniteSQLBuilder.selectGet(igniteMapping, dbFields);
+    String selectQuery = IgniteSQLBuilder.createSelectQueryGet(igniteMapping, dbFields);
     try (PreparedStatement stmt = connection.prepareStatement(selectQuery)) {
-      IgniteSQLBuilder.fillSelectStatement(stmt, igniteMapping, keyl);
+      IgniteSQLBuilder.fillSelectQuery(stmt, igniteMapping, keyl);
       ResultSet rs = stmt.executeQuery();
       boolean data = rs.next();
       T resp = null;
@@ -204,21 +207,20 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
     } catch (SQLException | IOException ex) {
       throw new GoraException(ex);
     }
-
   }
 
-  public T newInstance(ResultSet rs, String[] fields) throws GoraException, SQLException, IOException {
+  public T newInstance(ResultSet resultset, String[] fields) throws GoraException, SQLException, IOException {
     fields = getFieldsToQuery(fields);
     T persistent = newPersistent();
     for (String f : fields) {
       Schema.Field field = fieldMap.get(f);
       Schema fieldSchema = field.schema();
       String dbField = igniteMapping.getFields().get(f).getName();
-      Object sv = rs.getObject(dbField);
-      if (sv == null) {
+      Object fieldValue = resultset.getObject(dbField);
+      if (fieldValue == null) {
         continue;
       }
-      Object v = deserializeFieldValue(field, fieldSchema, sv, persistent);
+      Object v = deserializeFieldValue(field, fieldSchema, fieldValue, persistent);
       persistent.put(field.pos(), v);
       persistent.setDirty(field.pos());
     }
@@ -263,7 +265,6 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
         fieldValue = igniteValue;
     }
     return fieldValue;
-
   }
 
   @Override
@@ -280,17 +281,17 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
           //Composite keys pending..
         }
         for (Schema.Field field : fields) {
-          Column get = igniteMapping.getFields().get(field.name());
-          Object v = obj.get(field.pos());
-          if (get != null && v != null) {
+          Column mappedColumn = igniteMapping.getFields().get(field.name());
+          Object fieldValue = obj.get(field.pos());
+          if (mappedColumn != null && fieldValue != null) {
             Schema fieldSchema = field.schema();
-            Object serializedObj = serializeFieldValue(fieldSchema, v);
-            data.put(get, serializedObj);
+            Object serializedObj = serializeFieldValue(fieldSchema, fieldValue);
+            data.put(mappedColumn, serializedObj);
           }
         }
-        String baseInsertStatement = IgniteSQLBuilder.baseInsertStatement(igniteMapping, data);
+        String baseInsertStatement = IgniteSQLBuilder.createInsertQuery(igniteMapping, data);
         try (PreparedStatement stmt = connection.prepareStatement(baseInsertStatement)) {
-          IgniteSQLBuilder.fillInsertStatement(stmt, data);
+          IgniteSQLBuilder.fillInsertQuery(stmt, data);
           stmt.executeUpdate();
         } catch (SQLException ex) {
           throw new GoraException(ex);
@@ -307,15 +308,15 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
   @Override
   public boolean delete(K key) throws GoraException {
     String deleteQuery = null;
-    Object[] keyl = null;
+    Object[] keyArray = null;
     if (igniteMapping.getPrimaryKey().size() == 1) {
-      deleteQuery = IgniteSQLBuilder.delete(igniteMapping);
-      keyl = new Object[]{key};
+      deleteQuery = IgniteSQLBuilder.createDeleteQuery(igniteMapping);
+      keyArray = new Object[]{key};
     } else {
       //Composite key pending
     }
     try (PreparedStatement stmt = connection.prepareStatement(deleteQuery)) {
-      IgniteSQLBuilder.fillDeleteStatement(stmt, igniteMapping, keyl);
+      IgniteSQLBuilder.fillDeleteQuery(stmt, igniteMapping, keyArray);
       stmt.executeUpdate();
       return true;
     } catch (SQLException ex) {
@@ -325,20 +326,19 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
 
   @Override
   public long deleteByQuery(Query<K, T> query) throws GoraException {
-
     String deleteQuery;
     if (query.getFields() != null && query.getFields().length < igniteMapping.getFields().size()) {
       List<String> dbFields = new ArrayList<>();
       for (String af : query.getFields()) {
         dbFields.add(igniteMapping.getFields().get(af).getName());
       }
-      deleteQuery = IgniteSQLBuilder.deleteQueryFields(igniteMapping, dbFields);
+      deleteQuery = IgniteSQLBuilder.createDeleteQueryWithFields(igniteMapping, dbFields);
     } else {
-      deleteQuery = IgniteSQLBuilder.deleteQuery(igniteMapping);
+      deleteQuery = IgniteSQLBuilder.createDeleteQueryMultipleRecords(igniteMapping);
     }
-    String selectQueryWhere = IgniteSQLBuilder.queryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit());
+    String selectQueryWhere = IgniteSQLBuilder.createWhereClause(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit());
     try (PreparedStatement stmt = connection.prepareStatement(deleteQuery + selectQueryWhere)) {
-      IgniteSQLBuilder.fillSelectQuery(stmt, query.getStartKey(), query.getEndKey());
+      IgniteSQLBuilder.fillWhereClause(stmt, query.getStartKey(), query.getEndKey());
       stmt.executeUpdate();
       return 0;
     } catch (SQLException ex) {
@@ -351,19 +351,18 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
     String[] fields = getFieldsToQuery(query.getFields());
     //Avro fields to Ignite fields
     List<String> dbFields = new ArrayList<>();
-    for (String af : fields) {
-      dbFields.add(igniteMapping.getFields().get(af).getName());
+    for (String aField : fields) {
+      dbFields.add(igniteMapping.getFields().get(aField).getName());
     }
-    String selectQuery = IgniteSQLBuilder.selectQuery(igniteMapping, dbFields);
-    String selectQueryWhere = IgniteSQLBuilder.queryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit());
+    String selectQuery = IgniteSQLBuilder.createSelectQuery(igniteMapping, dbFields);
+    String selectQueryWhere = IgniteSQLBuilder.createWhereClause(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit());
     try (PreparedStatement stmt = connection.prepareStatement(selectQuery + selectQueryWhere)) {
       RowSetFactory factory = RowSetProvider.newFactory();
       CachedRowSet rowset = factory.createCachedRowSet();
-      IgniteSQLBuilder.fillSelectQuery(stmt, query.getStartKey(), query.getEndKey());
+      IgniteSQLBuilder.fillWhereClause(stmt, query.getStartKey(), query.getEndKey());
       ResultSet executeQuery = stmt.executeQuery();
       rowset.populate(executeQuery);
-      IgniteResult<K, T> igniteResult = new IgniteResult<>(this, query);
-      igniteResult.setResultSet(rowset);
+      IgniteResult<K, T> igniteResult = new IgniteResult<>(this, query, rowset);
       return igniteResult;
     } catch (SQLException ex) {
       throw new GoraException(ex);
@@ -462,7 +461,6 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
         break;
       default:
         throw new AssertionError(fieldSchema.getType().name());
-
     }
     return output;
   }
@@ -558,7 +556,6 @@ public class IgniteStore<K, T extends PersistentBase> extends DataStoreBase<K, T
       writer = new SpecificDatumWriter(fieldSchema);// ignore dirty bits
       writerMap.put(fieldSchema, writer);
     }
-
     return writer;
   }
 

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java
index 49959c8..e747477 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java
@@ -15,6 +15,6 @@
  * limitations under the License.
  */
 /**
- * This package contains all the Ignite store related classes.
+ * This package contains the core classes of the ignite datastore.
  */
-package org.apache.gora.ignite.store;
\ No newline at end of file
+package org.apache.gora.ignite.store;

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java
index 7459c67..9baa131 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java
@@ -22,15 +22,52 @@ package org.apache.gora.ignite.utils;
  */
 public class IgniteBackendConstants {
 
+  /**
+   * Ignite JDBC constants
+   */
+  public static final String DRIVER_NAME = "org.apache.ignite.IgniteJdbcThinDriver";
+  public static final String JDBC_PREFIX = "jdbc:ignite:thin://";
+
   /*
   * Default configurations for Ignite
-  */
+   */
   public static final String DEFAULT_IGNITE_HOST = "localhost";
   public static final String DEFAULT_IGNITE_PORT = "10800";
-  
+
   /*
    * A '42000' error code is thrown by Ignite when a non-existent table is queried.
    * More details: https://apacheignite-sql.readme.io/docs/jdbc-error-codes
    */
   public static final String DEFAULT_IGNITE_TABLE_NOT_EXISTS_CODE = "42000";
+
+  /**
+   * Property indicating the Ignite Schema to be used
+   */
+  public static final String PROP_SCHEMA = "gora.datastore.ignite.schema";
+
+  /**
+   * Property indicating the Ignite Cluster Node IP
+   */
+  public static final String PROP_HOST = "gora.datastore.ignite.host";
+
+  /**
+   * Property indicating the port used by the Ignite Server
+   */
+  public static final String PROP_PORT = "gora.datastore.ignite.port";
+
+  /**
+   * Property indicating the username to connect to the server
+   */
+  public static final String PROP_USER = "gora.datastore.ignite.user";
+
+  /**
+   * Property indicating the password to connect to the server
+   */
+  public static final String PROP_PASSWORD = "gora.datastore.ignite.password";
+
+  /**
+   * Property indicating additional JDBC options
+   */
+  public static final String PROP_ADDITIONALS = "gora.datastore.ignite.additionalConfigurations";
+
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java
index 3639b90..85c2960 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java
@@ -17,12 +17,26 @@
 package org.apache.gora.ignite.utils;
 
 import avro.shaded.com.google.common.collect.Lists;
+import com.healthmarketscience.sqlbuilder.BinaryCondition;
+import com.healthmarketscience.sqlbuilder.CreateTableQuery;
+import com.healthmarketscience.sqlbuilder.CustomSql;
+import com.healthmarketscience.sqlbuilder.DeleteQuery;
+import com.healthmarketscience.sqlbuilder.DropQuery;
+import com.healthmarketscience.sqlbuilder.InsertQuery;
+import com.healthmarketscience.sqlbuilder.SelectQuery;
+import com.healthmarketscience.sqlbuilder.SqlObject;
+import com.healthmarketscience.sqlbuilder.UpdateQuery;
+import com.healthmarketscience.sqlbuilder.custom.mysql.MysLimitClause;
+import com.healthmarketscience.sqlbuilder.dbspec.Constraint;
+import com.healthmarketscience.sqlbuilder.dbspec.basic.DbColumn;
+import com.healthmarketscience.sqlbuilder.dbspec.basic.DbConstraint;
+import com.healthmarketscience.sqlbuilder.dbspec.basic.DbSchema;
+import com.healthmarketscience.sqlbuilder.dbspec.basic.DbSpec;
+import com.healthmarketscience.sqlbuilder.dbspec.basic.DbTable;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
-import java.text.MessageFormat;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Locale;
 import java.util.Map;
 import java.util.Map.Entry;
 import org.apache.gora.ignite.store.Column;
@@ -34,11 +48,6 @@ import org.apache.gora.ignite.store.IgniteMapping;
  */
 public class IgniteSQLBuilder {
 
-  private static String format(String pattern, Object... args) {
-    MessageFormat messageFormat = new MessageFormat(pattern, Locale.getDefault());
-    return messageFormat.format(args);
-  }
-
   /**
    * Returns a SQL query for determine whether a table exists or not.
    *
@@ -46,7 +55,11 @@ public class IgniteSQLBuilder {
    * @return SQL query
    */
   public static String tableExists(String tableName) {
-    return format("SELECT * FROM {0} LIMIT 0", tableName);
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(tableName);
+    return new SelectQuery().addAllColumns().addFromTable(aTable)
+        .addCustomization(new MysLimitClause(0)).validate().toString();
   }
 
   /**
@@ -57,26 +70,24 @@ public class IgniteSQLBuilder {
    * @return SQL create query (DDL).
    */
   public static String createTable(IgniteMapping mapping) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("CREATE TABLE ");
-    sqlBuilder.append(mapping.getTableName());
-    sqlBuilder.append("(");
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
     ArrayList<Column> fieldsList = Lists.newArrayList(mapping.getPrimaryKey());
     fieldsList.addAll(Lists.newArrayList(mapping.getFields().values()));
     for (Column aColumn : fieldsList) {
       String name = aColumn.getName();
       Column.FieldType dataType = aColumn.getDataType();
-      sqlBuilder.append(name).append(" ").append(dataType.toString()).append(",");
+      aTable.addColumn(name, dataType.toString(), null);
     }
-    sqlBuilder.append("PRIMARY KEY ");
-    sqlBuilder.append("(");
+    String[] keys = new String[mapping.getPrimaryKey().size()];
     for (int i = 0; i < mapping.getPrimaryKey().size(); i++) {
-      sqlBuilder.append(mapping.getPrimaryKey().get(i).getName());
-      sqlBuilder.append(i == mapping.getPrimaryKey().size() - 1 ? "" : ",");
+      keys[i] = mapping.getPrimaryKey().get(i).getName();
     }
-    sqlBuilder.append(")");
-    sqlBuilder.append(");");
-    return sqlBuilder.toString();
+    aTable.addConstraint(new DbConstraint(aTable,
+        mapping.getTableName() + "_PRIMARY_KEY",
+        Constraint.Type.PRIMARY_KEY, keys));
+    return new CreateTableQuery(aTable, true).validate().toString();
   }
 
   /**
@@ -87,7 +98,8 @@ public class IgniteSQLBuilder {
    * @return SQL drop query (DDL).
    */
   public static String dropTable(String tableName) {
-    return format("DROP TABLE IF EXISTS {0} ;", tableName);
+    String statement = DropQuery.dropTable(tableName).validate().toString();
+    return statement.substring(0, 11) + "IF EXISTS " + statement.substring(11);
   }
 
   /**
@@ -98,25 +110,18 @@ public class IgniteSQLBuilder {
    * @param data A map containing the Column-Value pairs of the new record.
    * @return SQL insert statement
    */
-  public static String baseInsertStatement(IgniteMapping mapping, Map<Column, Object> data) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("MERGE INTO ");
-    sqlBuilder.append(mapping.getTableName());
-    sqlBuilder.append(" (");
+  public static String createInsertQuery(IgniteMapping mapping, Map<Column, Object> data) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    InsertQuery insertQuery = new InsertQuery(aTable);
     List<Entry<Column, Object>> list = new ArrayList<>(data.entrySet());
+    String[] columns = new String[list.size()];
     for (int i = 0; i < list.size(); i++) {
-      sqlBuilder.append(list.get(i).getKey().getName());
-      sqlBuilder.append(i == list.size() - 1 ? "" : ",");
-    }
-    sqlBuilder.append(")");
-    sqlBuilder.append(" VALUES ");
-    sqlBuilder.append(" (");
-    for (int i = 0; i < list.size(); i++) {
-      sqlBuilder.append("?");
-      sqlBuilder.append(i == list.size() - 1 ? "" : ",");
+      columns[i] = list.get(i).getKey().getName();
     }
-    sqlBuilder.append(" )");
-    return sqlBuilder.toString();
+    return insertQuery.addCustomPreparedColumns(columns).validate().toString()
+        .replaceFirst("INSERT", "MERGE");
   }
 
   /**
@@ -129,7 +134,7 @@ public class IgniteSQLBuilder {
    * @throws SQLException When invalid values are provided as parameters for the
    * insert statement.
    */
-  public static void fillInsertStatement(PreparedStatement statement, Map<Column, Object> insertData) throws SQLException {
+  public static void fillInsertQuery(PreparedStatement statement, Map<Column, Object> insertData) throws SQLException {
     List<Entry<Column, Object>> list = new ArrayList<>(insertData.entrySet());
     for (int i = 0; i < list.size(); i++) {
       int j = i + 1;
@@ -144,17 +149,17 @@ public class IgniteSQLBuilder {
    * @param mapping The ignite mapping definition of the data store
    * @return SQL delete statement
    */
-  public static String delete(IgniteMapping mapping) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("DELETE FROM ");
-    sqlBuilder.append(mapping.getTableName());
-    sqlBuilder.append(" WHERE ");
+  public static String createDeleteQuery(IgniteMapping mapping) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    DeleteQuery statement = new DeleteQuery(aTable);
     for (int i = 0; i < mapping.getPrimaryKey().size(); i++) {
-      sqlBuilder.append(mapping.getPrimaryKey().get(i).getName());
-      sqlBuilder.append("= ? ");
-      sqlBuilder.append(i == mapping.getPrimaryKey().size() - 1 ? "" : " AND ");
+      statement.addCondition(new BinaryCondition(BinaryCondition.Op.EQUAL_TO,
+          new DbColumn(aTable, mapping.getPrimaryKey().get(i).getName(), null),
+          SqlObject.QUESTION_MARK));
     }
-    return sqlBuilder.toString();
+    return statement.validate().toString();
   }
 
   /**
@@ -167,7 +172,7 @@ public class IgniteSQLBuilder {
    * record to be deleted
    * @throws SQLException When invalid keys' values are provided as parameters
    */
-  public static void fillDeleteStatement(PreparedStatement statement, IgniteMapping mapping, Object... deleteData) throws SQLException {
+  public static void fillDeleteQuery(PreparedStatement statement, IgniteMapping mapping, Object... deleteData) throws SQLException {
     assert mapping.getPrimaryKey().size() == deleteData.length;
     for (int i = 0; i < mapping.getPrimaryKey().size(); i++) {
       int j = i + 1;
@@ -183,22 +188,23 @@ public class IgniteSQLBuilder {
    * @param columns A list of columns to be retrieved within the select query
    * @return SQL select statement
    */
-  public static String selectGet(IgniteMapping mapping, List<String> columns) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("SELECT ");
+  public static String createSelectQueryGet(IgniteMapping mapping, List<String> columns) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    SelectQuery selectQuery = new SelectQuery();
+    selectQuery.addFromTable(aTable);
+    DbColumn[] lsColumns = new DbColumn[columns.size()];
     for (int i = 0; i < columns.size(); i++) {
-      sqlBuilder.append(columns.get(i));
-      sqlBuilder.append(i == columns.size() - 1 ? "" : " , ");
+      lsColumns[i] = aTable.addColumn(columns.get(i));
     }
-    sqlBuilder.append(" FROM ");
-    sqlBuilder.append(mapping.getTableName());
-    sqlBuilder.append(" WHERE ");
+    selectQuery.addColumns(lsColumns);
     for (int i = 0; i < mapping.getPrimaryKey().size(); i++) {
-      sqlBuilder.append(mapping.getPrimaryKey().get(i).getName());
-      sqlBuilder.append("= ? ");
-      sqlBuilder.append(i == mapping.getPrimaryKey().size() - 1 ? "" : " AND ");
+      selectQuery.addCondition(new BinaryCondition(BinaryCondition.Op.EQUAL_TO,
+          new DbColumn(aTable, mapping.getPrimaryKey().get(i).getName(), null),
+          SqlObject.QUESTION_MARK));
     }
-    return sqlBuilder.toString();
+    return selectQuery.validate().toString();
   }
 
   /**
@@ -211,7 +217,7 @@ public class IgniteSQLBuilder {
    * record to be retrieved
    * @throws SQLException When invalid keys' values are provided as parameters
    */
-  public static void fillSelectStatement(PreparedStatement statement, IgniteMapping mapping, Object... selectData) throws SQLException {
+  public static void fillSelectQuery(PreparedStatement statement, IgniteMapping mapping, Object... selectData) throws SQLException {
     assert mapping.getPrimaryKey().size() == selectData.length;
     for (int i = 0; i < mapping.getPrimaryKey().size(); i++) {
       int j = i + 1;
@@ -228,21 +234,23 @@ public class IgniteSQLBuilder {
    * query
    * @return SQL select statement
    */
-  public static String selectQuery(IgniteMapping mapping, List<String> selectFields) {
+  public static String createSelectQuery(IgniteMapping mapping, List<String> selectFields) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    SelectQuery selectQuery = new SelectQuery();
+    selectQuery.addFromTable(aTable);
     List<String> fields = new ArrayList<>();
     for (Column c : mapping.getPrimaryKey()) {
       fields.add(c.getName());
     }
     fields.addAll(selectFields);
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("SELECT ");
+    DbColumn[] lsColumns = new DbColumn[fields.size()];
     for (int i = 0; i < fields.size(); i++) {
-      sqlBuilder.append(fields.get(i));
-      sqlBuilder.append(i == fields.size() - 1 ? "" : " , ");
+      lsColumns[i] = aTable.addColumn(fields.get(i));
     }
-    sqlBuilder.append(" FROM ");
-    sqlBuilder.append(mapping.getTableName());
-    return sqlBuilder.toString();
+    selectQuery.addColumns(lsColumns);
+    return selectQuery.validate().toString();
   }
 
   /**
@@ -252,11 +260,12 @@ public class IgniteSQLBuilder {
    * @param mapping The ignite mapping definition of the data store
    * @return SQL delete statement
    */
-  public static String deleteQuery(IgniteMapping mapping) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("DELETE FROM ");
-    sqlBuilder.append(mapping.getTableName());
-    return sqlBuilder.toString();
+  public static String createDeleteQueryMultipleRecords(IgniteMapping mapping) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    DeleteQuery deleteQuery = new DeleteQuery(aTable);
+    return deleteQuery.validate().toString();
   }
 
   /**
@@ -267,19 +276,15 @@ public class IgniteSQLBuilder {
    * @param deleteFields A list of columns to be deleted (set to null)
    * @return SQL update statement
    */
-  public static String deleteQueryFields(IgniteMapping mapping, List<String> deleteFields) {
-    StringBuilder sqlBuilder = new StringBuilder();
-    sqlBuilder.append("UPDATE ");
-    sqlBuilder.append(mapping.getTableName());
-    if (!deleteFields.isEmpty()) {
-      sqlBuilder.append(" SET ");
-    }
+  public static String createDeleteQueryWithFields(IgniteMapping mapping, List<String> deleteFields) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    UpdateQuery updateQuery = new UpdateQuery(aTable);
     for (int i = 0; i < deleteFields.size(); i++) {
-      sqlBuilder.append(deleteFields.get(i));
-      sqlBuilder.append(" = null");
-      sqlBuilder.append(i == deleteFields.size() - 1 ? "" : " , ");
+      updateQuery.addSetClause(new DbColumn(aTable, deleteFields.get(i), null), SqlObject.NULL_VALUE);
     }
-    return sqlBuilder.toString();
+    return updateQuery.validate().toString();
   }
 
   /**
@@ -292,34 +297,34 @@ public class IgniteSQLBuilder {
    * @param limit The maximum number of records to be consider
    * @return SQL WHERE segment
    */
-  public static String queryWhere(IgniteMapping mapping, Object startKey, Object endKey, long limit) {
-    //composite keys pending
-    assert mapping.getPrimaryKey().size() == 1;
+  public static String createWhereClause(IgniteMapping mapping, Object startKey, Object endKey, long limit) {
+    DbSpec spec = new DbSpec();
+    DbSchema schema = spec.addDefaultSchema();
+    DbTable aTable = schema.addTable(mapping.getTableName());
+    SelectQuery selectQuery = new SelectQuery();
+    selectQuery.addFromTable(aTable);
+    String fisrtPart = selectQuery.validate().toString();
     String keycolumn = mapping.getPrimaryKey().get(0).getName();
-    StringBuilder sqlBuilder = new StringBuilder();
     if (startKey != null || endKey != null) {
-      sqlBuilder.append(" WHERE ");
       if (startKey != null && endKey != null && startKey.equals(endKey)) {
-        sqlBuilder.append(keycolumn);
-        sqlBuilder.append("= ?");
+        selectQuery.addCondition(new BinaryCondition(BinaryCondition.Op.EQUAL_TO,
+            new CustomSql(keycolumn), SqlObject.QUESTION_MARK));
       } else {
         if (startKey != null) {
-          sqlBuilder.append(keycolumn);
-          sqlBuilder.append(">= ?");
-        }
-        if (startKey != null && endKey != null) {
-          sqlBuilder.append(" AND ");
+          selectQuery.addCondition(new BinaryCondition(BinaryCondition.Op.GREATER_THAN_OR_EQUAL_TO,
+              new CustomSql(keycolumn), SqlObject.QUESTION_MARK));
         }
         if (endKey != null) {
-          sqlBuilder.append(keycolumn);
-          sqlBuilder.append("<= ?");
+          selectQuery.addCondition(new BinaryCondition(BinaryCondition.Op.LESS_THAN_OR_EQUAL_TO,
+              new CustomSql(keycolumn), SqlObject.QUESTION_MARK));
         }
       }
     }
     if (limit > 0) {
-      sqlBuilder.append(" LIMIT ").append(limit);
+      selectQuery.addCustomization(new MysLimitClause(limit));
     }
-    return sqlBuilder.toString();
+    String completeQuery = selectQuery.validate().toString();
+    return completeQuery.substring(fisrtPart.length());
   }
 
   /**
@@ -331,7 +336,7 @@ public class IgniteSQLBuilder {
    * @param endKey End key of the WHERE condition
    * @throws SQLException When invalid keys' values are provided as parameters
    */
-  public static void fillSelectQuery(PreparedStatement statement, Object startKey, Object endKey) throws SQLException {
+  public static void fillWhereClause(PreparedStatement statement, Object startKey, Object endKey) throws SQLException {
     if (startKey != null || endKey != null) {
       if (startKey != null && endKey != null && startKey.equals(endKey)) {
         statement.setObject(1, startKey);

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/main/java/org/apache/gora/ignite/utils/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/package-info.java
index ddba6e6..78fa1d0 100644
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/utils/package-info.java
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/package-info.java
@@ -15,6 +15,6 @@
  * limitations under the License.
  */
 /**
- * This package contains Ignite datastore related all classes.
+ * This package contains utility classes.
  */
 package org.apache.gora.ignite.utils;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/IgniteStoreMapReduceTest.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/IgniteStoreMapReduceTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/IgniteStoreMapReduceTest.java
new file mode 100644
index 0000000..2f82247
--- /dev/null
+++ b/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/IgniteStoreMapReduceTest.java
@@ -0,0 +1,65 @@
+/*
+ * 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.ignite.mapreduce;
+
+import java.io.IOException;
+import org.apache.gora.examples.generated.WebPage;
+import org.apache.gora.ignite.GoraIgniteTestDriver;
+import org.apache.gora.mapreduce.DataStoreMapReduceTestBase;
+import org.apache.gora.store.DataStore;
+import org.apache.gora.store.DataStoreFactory;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Executes tests for MR jobs over Ignite dataStore.
+ */
+public class IgniteStoreMapReduceTest extends DataStoreMapReduceTestBase {
+
+  private GoraIgniteTestDriver driver;
+
+  public IgniteStoreMapReduceTest() throws IOException {
+    super();
+    driver = new GoraIgniteTestDriver();
+  }
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    driver.setUpClass();
+    super.setUp();
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    super.tearDown();
+    driver.tearDownClass();
+  }
+
+  @Override
+  protected DataStore<String, WebPage> createWebPageDataStore() throws IOException {
+    try {
+      return DataStoreFactory.getDataStore(String.class, WebPage.class, driver.getConfiguration());
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/package-info.java b/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/package-info.java
new file mode 100644
index 0000000..6dd4be7
--- /dev/null
+++ b/gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/package-info.java
@@ -0,0 +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.
+ */
+/**
+ * This package contains map reduce tests.
+ */
+package org.apache.gora.ignite.mapreduce;

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java b/gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java
new file mode 100644
index 0000000..1899e5d
--- /dev/null
+++ b/gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java
@@ -0,0 +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.
+ */
+/**
+ * This package contains Ignite datastore test utilities.
+ */
+package org.apache.gora.ignite;

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java
new file mode 100644
index 0000000..03fc7e3
--- /dev/null
+++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java
@@ -0,0 +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.
+ */
+/**
+ * This package contains Ignite datastore tests.
+ */
+package org.apache.gora.ignite.store;

http://git-wip-us.apache.org/repos/asf/gora/blob/446f42ea/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index de30ec6..ddd0d14 100644
--- a/pom.xml
+++ b/pom.xml
@@ -433,7 +433,7 @@
           </dependencies>
         </plugin>
         <!--This plugin's configuration is used to store Eclipse m2e settings 
-          only. It has no influence on the Maven build itself. -->
+        only. It has no influence on the Maven build itself. -->
         <plugin>
           <groupId>org.eclipse.m2e</groupId>
           <artifactId>lifecycle-mapping</artifactId>
@@ -647,17 +647,17 @@
       <build>
         <plugins>
           <!-- <plugin> <groupId>org.apache.rat</groupId> <artifactId>apache-rat-plugin</artifactId> 
-            <version>${apache-rat-plugin.version}</version> <executions> <execution> 
-            <id>rat-verify</id> <phase>test</phase> <goals> <goal>check</goal> </goals> 
-            </execution> </executions> <configuration> <licenses> <license implementation="org.apache.rat.analysis.license.SimplePatternBasedLicense"> 
-            <licenseFamilyCategory>ASL20</licenseFamilyCategory> <licenseFamilyName>Apache 
-            Software License, 2.0</licenseFamilyName> <notes>Single licensed ASL v2.0</notes> 
-            <patterns> <pattern>Licensed to the Apache Software Foundation (ASF) under 
-            one or more contributor license agreements.</pattern> </patterns> </license> 
-            </licenses> <excludeSubProjects>false</excludeSubProjects> <excludes> <exclude>CHANGES.txt</exclude> 
-            <exclude>README.txt</exclude> <exclude>NOTICE.txt</exclude> <exclude>LICENSE.txt</exclude> 
-            <exclude>KEYS</exclude> <exclude>doap_Gora.rdf</exclude> <exclude>.gitignore/**/**</exclude> 
-            </excludes> </configuration> </plugin -->
+          <version>${apache-rat-plugin.version}</version> <executions> <execution> 
+          <id>rat-verify</id> <phase>test</phase> <goals> <goal>check</goal> </goals> 
+          </execution> </executions> <configuration> <licenses> <license implementation="org.apache.rat.analysis.license.SimplePatternBasedLicense"> 
+          <licenseFamilyCategory>ASL20</licenseFamilyCategory> <licenseFamilyName>Apache 
+          Software License, 2.0</licenseFamilyName> <notes>Single licensed ASL v2.0</notes> 
+          <patterns> <pattern>Licensed to the Apache Software Foundation (ASF) under 
+          one or more contributor license agreements.</pattern> </patterns> </license> 
+          </licenses> <excludeSubProjects>false</excludeSubProjects> <excludes> <exclude>CHANGES.txt</exclude> 
+          <exclude>README.txt</exclude> <exclude>NOTICE.txt</exclude> <exclude>LICENSE.txt</exclude> 
+          <exclude>KEYS</exclude> <exclude>doap_Gora.rdf</exclude> <exclude>.gitignore/**/**</exclude> 
+          </excludes> </configuration> </plugin -->
           <plugin>
             <groupId>org.apache.maven.plugins</groupId>
             <artifactId>maven-source-plugin</artifactId>
@@ -768,7 +768,7 @@
     <osgi.version>4.2.0</osgi.version>
     <!-- Avro Dependencies -->
     <avro.version>1.8.1</avro.version>
-<!--     <jackson.version>1.6.9</jackson.version> -->
+    <!--     <jackson.version>1.6.9</jackson.version> -->
     <!-- Hadoop Dependencies -->
     <hadoop-2.version>2.5.2</hadoop-2.version>
     <hadoop-2.test.version>2.5.2</hadoop-2.test.version>
@@ -782,6 +782,7 @@
     <cassandra.version>3.11.0</cassandra.version>
     <!-- Ignite Dependencies -->
     <ignite.version>2.6.0</ignite.version>
+    <sqlbuilder.version>2.1.7</sqlbuilder.version>
     <!-- Solr Dependencies -->
     <lucene-solr.version>6.5.1</lucene-solr.version>
     <solr-solrj.version>6.5.1</solr-solrj.version>
@@ -970,6 +971,11 @@
         <version>${project.version}</version>
         <type>test-jar</type>
       </dependency>
+      <dependency>
+        <groupId>com.healthmarketscience.sqlbuilder</groupId>
+        <artifactId>sqlbuilder</artifactId>
+        <version>${sqlbuilder.version}</version>
+      </dependency>
       
       <!-- End of Internal Depednecies -->
 
@@ -1045,26 +1051,26 @@
             <groupId>org.apache.hadoop</groupId>
             <artifactId>avro</artifactId>
           </exclusion>
-			    <exclusion>
-				    <artifactId>slf4j-log4j12</artifactId>
-				    <groupId>org.slf4j</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-common</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-yarn-common</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-mapreduce-client-core</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-auth</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
+          <exclusion>
+            <artifactId>slf4j-log4j12</artifactId>
+            <groupId>org.slf4j</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-common</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-yarn-common</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-auth</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
         </exclusions>
       </dependency>
       <dependency>
@@ -1077,26 +1083,26 @@
             <groupId>org.apache.hadoop</groupId>
             <artifactId>avro</artifactId>
           </exclusion>
-			    <exclusion>
-				    <artifactId>slf4j-log4j12</artifactId>
-				    <groupId>org.slf4j</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-common</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-yarn-common</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-mapreduce-client-core</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
-			    <exclusion>
-				    <artifactId>hadoop-auth</artifactId>
-				    <groupId>org.apache.hadoop</groupId>
-			    </exclusion>
+          <exclusion>
+            <artifactId>slf4j-log4j12</artifactId>
+            <groupId>org.slf4j</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-common</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-yarn-common</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-mapreduce-client-core</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
+          <exclusion>
+            <artifactId>hadoop-auth</artifactId>
+            <groupId>org.apache.hadoop</groupId>
+          </exclusion>
         </exclusions>
       </dependency>
 
@@ -1595,6 +1601,11 @@
         <artifactId>ignite-indexing</artifactId>
         <version>${ignite.version}</version>
       </dependency>
+      <dependency>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-core</artifactId>
+        <version>${ignite.version}</version>
+      </dependency>
     </dependencies>
   </dependencyManagement>