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:40 UTC

[05/11] gora git commit: Add mapping reading and initialization for Ignite backend

Add mapping reading and initialization for Ignite backend

Basic configuration and connection classes were implemented for the
Ignite backend. The basic skeleton of the new DataStore was initialized
using mapping files and some trivial methods were implemented
(#close(), #flush(), #getSchemaName(), ...). Also, an embembed Ignite
instance was included within the test packages in order to be used in
testing.


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

Branch: refs/heads/master
Commit: 7545170df8047ea6127660c1750e1d0b59044e49
Parents: 7d67137
Author: Carlos M <ca...@gmail.com>
Authored: Wed Jun 6 00:03:10 2018 -0500
Committer: Carlos M <ca...@gmail.com>
Committed: Wed Jun 6 00:08:48 2018 -0500

----------------------------------------------------------------------
 gora-ignite/pom.xml                             |   21 +-
 .../gora/ignite/encoders/BinaryEncoder.java     |  191 ----
 .../apache/gora/ignite/encoders/Encoder.java    |   72 --
 .../apache/gora/ignite/encoders/HexEncoder.java |  204 ----
 .../ignite/encoders/SignedBinaryEncoder.java    |  110 --
 .../org/apache/gora/ignite/encoders/Utils.java  |   91 --
 .../gora/ignite/encoders/package-info.java      |   20 -
 .../apache/gora/ignite/query/IgniteQuery.java   |   45 -
 .../apache/gora/ignite/query/IgniteResult.java  |  101 --
 .../apache/gora/ignite/query/package-info.java  |   21 -
 .../org/apache/gora/ignite/store/Column.java    |   70 ++
 .../apache/gora/ignite/store/IgniteMapping.java |   44 +-
 .../gora/ignite/store/IgniteMappingBuilder.java |  107 ++
 .../gora/ignite/store/IgniteParameters.java     |  139 +++
 .../apache/gora/ignite/store/IgniteStore.java   | 1013 ++----------------
 .../ignite/util/FixedByteArrayOutputStream.java |   45 -
 .../apache/gora/ignite/util/package-info.java   |   20 -
 .../gora/ignite/GoraIgniteTestDriver.java       |   49 +-
 .../org/apache/gora/ignite/package-info.java    |   21 -
 .../ignite/store/AuthenticationTokenTest.java   |   90 --
 .../gora/ignite/store/IgniteStoreTest.java      |   87 --
 .../apache/gora/ignite/store/PartitionTest.java |   96 --
 .../gora/ignite/store/TestIgniteStore.java      |   32 +
 .../apache/gora/ignite/store/package-info.java  |   21 -
 .../apache/gora/ignite/util/HexEncoderTest.java |   56 -
 .../ignite/util/SignedBinaryEncoderTest.java    |  167 ---
 .../apache/gora/ignite/util/package-info.java   |   20 -
 .../test/resources/gora-accumulo-mapping.xml    |   59 -
 .../src/test/resources/gora-ignite-mapping.xml  |   42 +
 gora-ignite/src/test/resources/gora.properties  |   11 +-
 nbactions.xml                                   |   13 +
 31 files changed, 550 insertions(+), 2528 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/pom.xml
----------------------------------------------------------------------
diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml
index ddd3a4a..fa31195 100644
--- a/gora-ignite/pom.xml
+++ b/gora-ignite/pom.xml
@@ -50,7 +50,7 @@
   </ciManagement>
 
   <properties>
-    <ignite.version>1.7.1</ignite.version>
+    <ignite.version>2.4.0</ignite.version>
     <osgi.import>*</osgi.import>
     <osgi.export>org.apache.gora.ignite*;version="${project.version}";-noimport:=true</osgi.export>
   </properties>
@@ -115,13 +115,6 @@
     </dependency>
 
     <dependency>
-      <groupId>org.apache.ignite</groupId>
-      <artifactId>ignite-minicluster</artifactId>
-      <version>${ignite.version}</version>
-      <scope>test</scope>
-    </dependency>
-
-    <dependency>
       <groupId>commons-io</groupId>
       <artifactId>commons-io</artifactId>
     </dependency>
@@ -144,6 +137,12 @@
       <groupId>org.apache.avro</groupId>
       <artifactId>avro</artifactId>
     </dependency>
+    
+    <dependency>
+      <groupId>org.jdom</groupId>
+      <artifactId>jdom</artifactId>
+      <scope>compile</scope>
+    </dependency>
 
     <!-- Logging Dependencies -->
     <dependency>
@@ -171,6 +170,12 @@
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-minicluster</artifactId>
     </dependency>
+    
+    <dependency>
+      <groupId>org.apache.ignite</groupId>
+      <artifactId>ignite-indexing</artifactId>
+      <version>2.4.0</version>
+    </dependency>
 
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java
deleted file mode 100644
index ce6eded..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java
+++ /dev/null
@@ -1,191 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.encoders;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.apache.gora.ignite.util.FixedByteArrayOutputStream;
-
-/**
- * 
- */
-public class BinaryEncoder implements Encoder {
-
-  @Override
-  public byte[] encodeShort(short s) throws IOException {
-    return encodeShort(s, new byte[2]);
-  }
-
-  @Override
-  public byte[] encodeShort(short s, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      dos.writeShort(s);
-      dos.close();
-      return ret;
-    }
-  }
-
-  @Override
-  public short decodeShort(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      short s = dis.readShort();
-      dis.close();
-      return s;
-    }
-  }
-
-  @Override
-  public byte[] encodeInt(int i) throws IOException {
-    return encodeInt(i, new byte[4]);
-  }
-
-  @Override
-  public byte[] encodeInt(int i, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      dos.writeInt(i);
-      dos.close();
-      return ret;
-    }
-  }
-
-  @Override
-  public int decodeInt(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      int i = dis.readInt();
-      dis.close();
-      return i;
-    }
-  }
-
-  @Override
-  public byte[] encodeLong(long l) throws IOException {
-    return encodeLong(l, new byte[8]);
-  }
-
-  @Override
-  public byte[] encodeLong(long l, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      dos.writeLong(l);
-      dos.close();
-      return ret;
-    }
-  }
-
-  @Override
-  public long decodeLong(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      long l = dis.readLong();
-      dis.close();
-      return l;
-    }
-  }
-
-  @Override
-  public byte[] encodeDouble(double d) throws IOException {
-    return encodeDouble(d, new byte[8]);
-  }
-
-  @Override
-  public byte[] encodeDouble(double d, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      long l = Double.doubleToRawLongBits(d);
-      dos.writeLong(l);
-      dos.close();
-      return ret;
-    }
-  }
-
-  @Override
-  public double decodeDouble(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      long l = dis.readLong();
-      dis.close();
-      return Double.longBitsToDouble(l);
-    }
-  }
-
-  @Override
-  public byte[] encodeFloat(float d) throws IOException {
-    return encodeFloat(d, new byte[4]);
-  }
-
-  @Override
-  public byte[] encodeFloat(float f, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      int i = Float.floatToRawIntBits(f);
-      dos.writeInt(i);
-      return ret;
-    }
-  }
-
-  @Override
-  public float decodeFloat(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      int i = dis.readInt();
-      return Float.intBitsToFloat(i);
-    }
-  }
-
-  @Override
-  public byte[] encodeByte(byte b, byte[] ret) {
-    ret[0] = 0;
-    return ret;
-  }
-
-  @Override
-  public byte[] encodeByte(byte b) {
-    return encodeByte(b, new byte[1]);
-  }
-
-  @Override
-  public byte decodeByte(byte[] a) {
-    return a[0];
-  }
-
-  @Override
-  public boolean decodeBoolean(byte[] a) throws IOException {
-    try (DataInputStream dis = new DataInputStream(new ByteArrayInputStream(a))){
-      return dis.readBoolean();
-    }
-  }
-
-  @Override
-  public byte[] encodeBoolean(boolean b) throws IOException {
-    return encodeBoolean(b, new byte[1]);
-  }
-
-  @Override
-  public byte[] encodeBoolean(boolean b, byte[] ret) throws IOException {
-    try (DataOutputStream dos = new DataOutputStream(new FixedByteArrayOutputStream(ret))){
-      dos.writeBoolean(b);
-      return ret;
-    }
-  }
-
-  @Override
-  public byte[] lastPossibleKey(int size, byte[] er) {
-    return Utils.lastPossibleKey(size, er);
-  }
-
-  @Override
-  public byte[] followingKey(int size, byte[] per) {
-    return Utils.followingKey(size, per);
-  }
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java
deleted file mode 100644
index deea4a7..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.encoders;
-
-import java.io.IOException;
-
-/**
- * 
- */
-public interface Encoder {
-  
-  public byte[] encodeByte(byte b, byte[] ret);
-  
-  public byte[] encodeByte(byte b);
-  
-  public byte decodeByte(byte[] a);
-
-  public byte[] encodeShort(short s) throws IOException;
-  
-  public byte[] encodeShort(short s, byte[] ret) throws IOException;
-  
-  public short decodeShort(byte[] a) throws IOException;
-  
-  public byte[] encodeInt(int i) throws IOException;
-  
-  public byte[] encodeInt(int i, byte[] ret) throws IOException;
-  
-  public int decodeInt(byte[] a) throws IOException;
-  
-  public byte[] encodeLong(long l) throws IOException;
-  
-  public byte[] encodeLong(long l, byte[] ret) throws IOException;
-  
-  public long decodeLong(byte[] a) throws IOException;
-  
-  public byte[] encodeDouble(double d) throws IOException;
-  
-  public byte[] encodeDouble(double d, byte[] ret) throws IOException;
-  
-  public double decodeDouble(byte[] a) throws IOException;
-  
-  public byte[] encodeFloat(float d) throws IOException;
-  
-  public byte[] encodeFloat(float f, byte[] ret) throws IOException;
-  
-  public float decodeFloat(byte[] a) throws IOException;
-  
-  public boolean decodeBoolean(byte[] val) throws IOException;
-  
-  public byte[] encodeBoolean(boolean b) throws IOException;
-  
-  public byte[] encodeBoolean(boolean b, byte[] ret) throws IOException;
-
-  byte[] followingKey(int size, byte[] per);
-
-  byte[] lastPossibleKey(int size, byte[] er);
-
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java
deleted file mode 100644
index 8568ba9..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java
+++ /dev/null
@@ -1,204 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.encoders;
-
-/**
- * Encodes data in a ascii hex representation
- */
-
-public class HexEncoder implements Encoder {
-  
-  private byte[] chars = new byte[] {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'a', 'b', 'c', 'd', 'e', 'f'};
-
-  private void encode(byte[] a, long l) {
-    for (int i = a.length - 1; i >= 0; i--) {
-      a[i] = chars[(int) (l & 0x0f)];
-      l = l >>> 4;
-    }
-  }
-
-  private int fromChar(byte b) {
-    if (b >= '0' && b <= '9') {
-      return b - '0';
-    } else if (b >= 'a' && b <= 'f') {
-      return b - 'a' + 10;
-    }
-    
-    throw new IllegalArgumentException("Bad char " + b);
-  }
-  
-  private long decode(byte[] a) {
-    long b = 0;
-    for (byte anA : a) {
-      b = b << 4;
-      b |= fromChar(anA);
-    }
-    
-    return b;
-  }
-
-  @Override
-  public byte[] encodeByte(byte b, byte[] ret) {
-    encode(ret, 0xff & b);
-    return ret;
-  }
-  
-  @Override
-  public byte[] encodeByte(byte b) {
-    return encodeByte(b, new byte[2]);
-  }
-  
-  @Override
-  public byte decodeByte(byte[] a) {
-    return (byte) decode(a);
-  }
-  
-  @Override
-  public byte[] encodeShort(short s) {
-    return encodeShort(s, new byte[4]);
-  }
-  
-  @Override
-  public byte[] encodeShort(short s, byte[] ret) {
-    encode(ret, 0xffff & s);
-    return ret;
-  }
-  
-  @Override
-  public short decodeShort(byte[] a) {
-    return (short) decode(a);
-  }
-  
-  @Override
-  public byte[] encodeInt(int i) {
-    return encodeInt(i, new byte[8]);
-  }
-  
-  @Override
-  public byte[] encodeInt(int i, byte[] ret) {
-    encode(ret, i);
-    return ret;
-  }
-  
-  @Override
-  public int decodeInt(byte[] a) {
-    return (int) decode(a);
-  }
-  
-  @Override
-  public byte[] encodeLong(long l) {
-    return encodeLong(l, new byte[16]);
-  }
-  
-  @Override
-  public byte[] encodeLong(long l, byte[] ret) {
-    encode(ret, l);
-    return ret;
-  }
-  
-  @Override
-  public long decodeLong(byte[] a) {
-    return decode(a);
-  }
-  
-  @Override
-  public byte[] encodeDouble(double d) {
-    return encodeDouble(d, new byte[16]);
-  }
-  
-  @Override
-  public byte[] encodeDouble(double d, byte[] ret) {
-    return encodeLong(Double.doubleToRawLongBits(d), ret);
-  }
-  
-  @Override
-  public double decodeDouble(byte[] a) {
-    return Double.longBitsToDouble(decodeLong(a));
-  }
-  
-  @Override
-  public byte[] encodeFloat(float d) {
-    return encodeFloat(d, new byte[16]);
-  }
-  
-  @Override
-  public byte[] encodeFloat(float d, byte[] ret) {
-    return encodeInt(Float.floatToRawIntBits(d), ret);
-  }
-  
-  @Override
-  public float decodeFloat(byte[] a) {
-    return Float.intBitsToFloat(decodeInt(a));
-  }
-  
-  @Override
-  public boolean decodeBoolean(byte[] val) {
-      return decodeByte(val) == 1;
-  }
-  
-  @Override
-  public byte[] encodeBoolean(boolean b) {
-    return encodeBoolean(b, new byte[2]);
-  }
-  
-  @Override
-  public byte[] encodeBoolean(boolean b, byte[] ret) {
-    if (b)
-      encode(ret, 1);
-    else
-      encode(ret, 0);
-    
-    return ret;
-  }
-  
-  private byte[] toBinary(byte[] hex) {
-    byte[] bin = new byte[(hex.length / 2) + (hex.length % 2)];
-    
-    int j = 0;
-    for (int i = 0; i < bin.length; i++) {
-      bin[i] = (byte) (fromChar(hex[j++]) << 4);
-      if (j >= hex.length)
-        break;
-      bin[i] |= (byte) fromChar(hex[j++]);
-    }
-    
-    return bin;
-  }
-  
-  private byte[] fromBinary(byte[] bin) {
-    byte[] hex = new byte[bin.length * 2];
-    
-    int j = 0;
-    for (byte aBin : bin) {
-      hex[j++] = chars[0x0f & (aBin >>> 4)];
-      hex[j++] = chars[0x0f & aBin];
-    }
-    
-    return hex;
-  }
-
-  @Override
-  public byte[] followingKey(int size, byte[] per) {
-    return fromBinary(Utils.followingKey(size, toBinary(per)));
-  }
-  
-  @Override
-  public byte[] lastPossibleKey(int size, byte[] er) {
-    return fromBinary(Utils.lastPossibleKey(size, toBinary(er)));
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java
deleted file mode 100644
index a8216f4..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java
+++ /dev/null
@@ -1,110 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.encoders;
-
-import java.io.IOException;
-
-/**
- * This class transforms this bits within a primitive type so that 
- * the bit representation sorts correctly lexographicaly. Primarily 
- * it does some simple transformations so that negative numbers sort 
- * before positive numbers, when compared lexographically.
- */
-public class SignedBinaryEncoder extends BinaryEncoder {
-
-  @Override
-  public byte[] encodeShort(short s, byte[] ret) throws IOException{
-    s = (short)((s & 0xffff) ^ 0x8000);
-    return super.encodeShort(s, ret);
-  }
-
-  @Override
-  public short decodeShort(byte[] a) throws IOException{
-    short s = super.decodeShort(a);
-    s = (short)((s & 0xffff) ^ 0x8000);
-    return s;
-  }
-
-  @Override
-  public byte[] encodeInt(int i, byte[] ret) throws IOException{
-    i = i ^ 0x80000000;
-    return super.encodeInt(i, ret);
-  }
-
-  @Override
-  public int decodeInt(byte[] a) throws IOException{
-    int i = super.decodeInt(a);
-    i = i ^ 0x80000000;
-    return i;
-  }
-
-  @Override
-  public byte[] encodeLong(long l, byte[] ret) throws IOException{
-    l = l ^ 0x8000000000000000L;
-    return super.encodeLong(l, ret);
-  }
-
-  @Override
-  public long decodeLong(byte[] a) throws IOException {
-    long l = super.decodeLong(a);
-    l = l ^ 0x8000000000000000L;
-    return l;
-  }
-
-  @Override
-  public byte[] encodeDouble(double d, byte[] ret) throws IOException {
-    long l = Double.doubleToRawLongBits(d);
-    if(l < 0)
-      l = ~l;
-    else
-      l = l ^ 0x8000000000000000L;
-    return super.encodeLong(l,ret);
-  }
-
-  @Override
-  public double decodeDouble(byte[] a) throws IOException{
-    long l = super.decodeLong(a);
-    if(l < 0)
-      l = l ^ 0x8000000000000000L;
-    else
-      l = ~l;
-    return Double.longBitsToDouble(l);
-  }
-
-  @Override
-  public byte[] encodeFloat(float f, byte[] ret) throws IOException {
-    int i = Float.floatToRawIntBits(f);
-    if(i < 0)
-      i = ~i;
-    else
-      i = i ^ 0x80000000;
-
-    return super.encodeInt(i, ret);
-
-  }
-
-  @Override
-  public float decodeFloat(byte[] a) throws IOException{
-    int i = super.decodeInt(a);
-    if(i < 0)
-      i = i ^ 0x80000000;
-    else
-      i = ~i;
-    return Float.intBitsToFloat(i);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java
deleted file mode 100644
index 8a5980c..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.encoders;
-
-import java.math.BigInteger;
-import java.util.Arrays;
-
-/**
- * 
- */
-public class Utils {
-  private static BigInteger newPositiveBigInteger(byte[] er) {
-    byte[] copy = new byte[er.length + 1];
-    System.arraycopy(er, 0, copy, 1, er.length);
-    BigInteger bi = new BigInteger(copy);
-    return bi;
-  }
-  
-  public static byte[] lastPossibleKey(int size, byte[] er) {
-    if (size == er.length)
-      return er;
-    
-    if (er.length > size)
-      throw new IllegalArgumentException();
-    
-    BigInteger bi = newPositiveBigInteger(er);
-    if (bi.equals(BigInteger.ZERO))
-      throw new IllegalArgumentException("Nothing comes before zero");
-    
-    bi = bi.subtract(BigInteger.ONE);
-    
-    byte[] ret = new byte[size];
-    Arrays.fill(ret, (byte) 0xff);
-    
-    System.arraycopy(getBytes(bi, er.length), 0, ret, 0, er.length);
-    
-    return ret;
-  }
-  
-  private static byte[] getBytes(BigInteger bi, int minLen) {
-    byte[] ret = bi.toByteArray();
-    
-    if (ret[0] == 0) {
-      // remove leading 0 that makes num positive
-      byte[] copy = new byte[ret.length - 1];
-      System.arraycopy(ret, 1, copy, 0, copy.length);
-      ret = copy;
-    }
-    
-    // leading digits are dropped
-    byte[] copy = new byte[minLen];
-    if (bi.compareTo(BigInteger.ZERO) < 0) {
-      Arrays.fill(copy, (byte) 0xff);
-    }
-    System.arraycopy(ret, 0, copy, minLen - ret.length, ret.length);
-    
-    return copy;
-  }
-  
-  public static byte[] followingKey(int size, byte[] per) {
-    
-    if (per.length > size)
-      throw new IllegalArgumentException();
-    
-    if (size == per.length) {
-      // add one
-      BigInteger bi = new BigInteger(per);
-      bi = bi.add(BigInteger.ONE);
-      if (bi.equals(BigInteger.ZERO)) {
-        throw new IllegalArgumentException("Wrapped");
-      }
-      return getBytes(bi, size);
-    } else {
-      return Arrays.copyOf(per, size);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java
----------------------------------------------------------------------
diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java
deleted file mode 100644
index 574aa24..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * This package contains Ignite store related util classes for encoder.
- */
-package org.apache.gora.ignite.encoders;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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
deleted file mode 100644
index 85a59c9..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.query;
-
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.query.impl.QueryBase;
-import org.apache.gora.store.DataStore;
-
-/**
- *  Ignite specific implementation of the {@link org.apache.gora.query.Query} interface.
- */
-public class IgniteQuery<K,T extends PersistentBase> extends QueryBase<K,T> {
-
-  /**
-   * Constructor for the query
-   */
-  public IgniteQuery() {
-    super(null);
-  }
-
-  /**
-   * Constructor for the query
-   *
-   * @param dataStore Data store used
-   *
-   */
-  public IgniteQuery(DataStore<K,T> dataStore) {
-    super(dataStore);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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
deleted file mode 100644
index 416e650..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.gora.ignite.query;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Map.Entry;
-
-import org.apache.ignite.core.client.RowIterator;
-import org.apache.ignite.core.client.Scanner;
-import org.apache.ignite.core.data.ByteSequence;
-import org.apache.ignite.core.data.Key;
-import org.apache.ignite.core.data.Value;
-import org.apache.gora.ignite.store.IgniteStore;
-import org.apache.gora.persistency.impl.PersistentBase;
-import org.apache.gora.query.Query;
-import org.apache.gora.query.impl.ResultBase;
-import org.apache.gora.store.DataStore;
-
-/**
- * Ignite specific implementation of the {@link org.apache.gora.query.Result} interface.
- */
-public class IgniteResult<K,T extends PersistentBase> extends ResultBase<K,T> {
-  
-  private RowIterator iterator;
-
-  /**
-   * Gets the data store used
-   */
-  public IgniteStore<K,T> getDataStore() {
-    return (IgniteStore<K,T>) super.getDataStore();
-  }
-
-  /**
-   * @param dataStore
-   * @param query
-   * @param scanner
-   */
-  public IgniteResult(DataStore<K,T> dataStore, Query<K,T> query, Scanner scanner) {
-    super(dataStore, query);
-    
-    if (this.limit > 0) {
-      scanner.setBatchSize((int) this.limit);
-    }
-    iterator = new RowIterator(scanner.iterator());
-  }
-
-  /**
-   * Gets the items reading progress
-   */
-  @Override
-  public float getProgress() throws IOException {
-    if (this.limit != -1) {
-      return (float) this.offset / (float) this.limit;
-    } else {
-      return 0;
-    }
-  }
-  
-  @Override
-  public void close() throws IOException {
-    
-  }
-
-  /**
-   * Gets the next item
-   */
-  @Override
-  protected boolean nextInner() throws IOException {
-    
-    if (!iterator.hasNext())
-      return false;
-    
-    key = null;
-    
-    Iterator<Entry<Key,Value>> nextRow = iterator.next();
-    ByteSequence row = getDataStore().populate(nextRow, persistent);
-    key = ((IgniteStore<K, T>) dataStore).fromBytes(getKeyClass(), row.toArray());
-    
-    return true;
-  }
-
-  @Override
-  public int size() {
-    return (int) this.limit;
-  }
-}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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
deleted file mode 100644
index b1a306c..0000000
--- a/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- * This package contains all the Ignite store query representation class as well as Result set representing class
- * when query is executed over the Ignite dataStore.
- */
-package org.apache.gora.ignite.query;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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
new file mode 100644
index 0000000..d06591e
--- /dev/null
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java
@@ -0,0 +1,70 @@
+/**
+ * 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.store;
+
+public class Column {
+
+  private String name;
+  private FieldType dataType;
+
+  public Column(String name, FieldType dataType) {
+    this.name = name;
+    this.dataType = dataType;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  public FieldType getDataType() {
+    return dataType;
+  }
+
+  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
+   * https://apacheignite-sql.readme.io/docs/data-types
+   */
+  public static enum FieldType {
+    BOOLEAN,
+    INT,
+    TINYINT,
+    SMALLINT,
+    BIGINT,
+    DECIMAL,
+    DOUBLE,
+    REAL,
+    TIME,
+    DATE,
+    TIMESTAMP,
+    VARCHAR,
+    CHAR,
+    UUID,
+    BINARY,
+    ARRAY
+  }
+}

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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 b46c063..2e59da1 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
@@ -17,28 +17,44 @@
 package org.apache.gora.ignite.store;
 
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
-import org.apache.ignite.core.util.Pair;
-import org.apache.hadoop.io.Text;
-
 /**
  * Mapping definitions for Ignite.
  */
 public class IgniteMapping {
 
-  /**
-   * A map of field names to Field objects containing schema's fields
-   */
-  Map<String,Pair<Text,Text>> fieldMap = new HashMap<>();
+  private String tableName;
+  private Map<String, Column> fields;
+  private List<String> primaryKey;
+
+  public IgniteMapping() {
+    fields = new HashMap<>();
+  }
+
+  public String getTableName() {
+    return tableName;
+  }
+
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  public Map<String, Column> getFields() {
+    return fields;
+  }
+
+  public void setFields(Map<String, Column> fields) {
+    this.fields = fields;
+  }
 
-  /**
-   * Look up the column associated to the Avro field.
-   */
-  Map<Pair<Text,Text>,String> columnMap = new HashMap<>();
+  public List<String> getPrimaryKey() {
+    return primaryKey;
+  }
 
-  Map<String,String> tableConfig = new HashMap<>();
-  String tableName;
-  String encoder;
+  public void setPrimaryKey(List<String> primaryKey) {
+    this.primaryKey = primaryKey;
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/gora/blob/7545170d/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
new file mode 100644
index 0000000..81a3fae
--- /dev/null
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java
@@ -0,0 +1,107 @@
+/**
+ * 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.store;
+
+import com.google.inject.ConfigurationException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.gora.persistency.impl.PersistentBase;
+import org.jdom.Document;
+import org.jdom.Element;
+import org.jdom.JDOMException;
+import org.jdom.input.SAXBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Builder for Mapping definitions of Ignite.
+ */
+public class IgniteMappingBuilder<K, T extends PersistentBase> {
+
+  private static final Logger LOG = LoggerFactory.getLogger(IgniteMappingBuilder.class);
+  /**
+   * Mapping instance being built
+   */
+  private IgniteMapping igniteMapping;
+
+  private final IgniteStore<K, T> dataStore;
+
+  public IgniteMappingBuilder(final IgniteStore<K, T> store) {
+    this.igniteMapping = new IgniteMapping();
+    this.dataStore = store;
+  }
+
+  public IgniteMapping getIgniteMapping() {
+    return igniteMapping;
+  }
+
+  public void setIgniteMapping(IgniteMapping igniteMapping) {
+    this.igniteMapping = igniteMapping;
+  }
+
+  public void readMappingFile(String mappingFile) {
+    try {
+      SAXBuilder saxBuilder = new SAXBuilder();
+      InputStream inputStream = getClass().getClassLoader().getResourceAsStream(mappingFile);
+      if (inputStream == null) {
+        LOG.error("Mapping file '{}' could not be found!", mappingFile);
+        throw new IOException("Mapping file '" + mappingFile + "' could not be found!");
+      }
+      Document document = saxBuilder.build(inputStream);
+      if (document == null) {
+        LOG.error("Mapping file '{}' could not be found!", mappingFile);
+        throw new IOException("Mapping file '" + mappingFile + "' could not be found!");
+      }
+      List<Element> classes = document.getRootElement().getChildren("class");
+      for (Element classElement : classes) {
+        if (classElement.getAttributeValue("keyClass").equals(
+            dataStore.getKeyClass().getCanonicalName())
+            && classElement.getAttributeValue("name").equals(
+                dataStore.getPersistentClass().getCanonicalName())) {
+          final String tableNameFromMapping = classElement.getAttributeValue("table");
+          String tableName = dataStore.getSchemaName(tableNameFromMapping, dataStore.getPersistentClass());
+          igniteMapping.setTableName(tableName);
+          List<Element> prColumns = classElement.getChildren("primarykey");
+          List<String> prFields = new ArrayList<>();
+          for (Element aPrimaryKey : prColumns) {
+            prFields.add(aPrimaryKey.getAttributeValue("column"));
+          }
+          igniteMapping.setPrimaryKey(prFields);
+          List<Element> fields = classElement.getChildren("field");
+          Map<String, Column> mp = new HashMap<>();
+          for (Element field : fields) {
+            String fieldName = field.getAttributeValue("name");
+            String columnName = field.getAttributeValue("column");
+            String columnType = field.getAttributeValue("type");
+            mp.put(fieldName, new Column(columnName, Column.FieldType.valueOf(columnType)));
+          }
+          igniteMapping.setFields(mp);
+          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/7545170d/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
new file mode 100644
index 0000000..15b5bae
--- /dev/null
+++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java
@@ -0,0 +1,139 @@
+/*
+ * Copyright 2018 The Apache Software Foundation.
+ *
+ * Licensed 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.store;
+
+import java.util.Properties;
+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;
+  private String user;
+  private String password;
+  private String additionalConfigurations;
+
+  /**
+   *
+   * @param host
+   * @param port Optional port for Ignite Server
+   * @param user Optional username for Ignite
+   * @param password Optional password for Ignite
+   * @param additionalConfigurations Optional additional configurations for
+   * Ignite
+   */
+  private IgniteParameters(String host, String port, String schema, String user, String password, String additionalConfigurations) {
+    this.host = host;
+    this.port = port;
+    this.schema = schema;
+    this.user = user;
+    this.password = password;
+    this.additionalConfigurations = additionalConfigurations;
+  }
+
+  public String getHost() {
+    return host;
+  }
+
+  public void setHost(String host) {
+    this.host = host;
+  }
+
+  public String getPort() {
+    return port;
+  }
+
+  public void setPort(String port) {
+    this.port = port;
+  }
+
+  public String getUser() {
+    return user;
+  }
+
+  public void setUser(String user) {
+    this.user = user;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  public void setPassword(String password) {
+    this.password = password;
+  }
+
+  public String getAdditionalConfigurations() {
+    return additionalConfigurations;
+  }
+
+  public void setAdditionalConfigurations(String additionalConfigurations) {
+    this.additionalConfigurations = additionalConfigurations;
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public void setSchema(String schema) {
+    this.schema = schema;
+  }
+
+  public static IgniteParameters load(Properties properties, Configuration conf) {
+    return new IgniteParameters(
+        properties.getProperty(PROP_HOST, "localhost"),
+        properties.getProperty(PROP_PORT, "10800"),
+        properties.getProperty(PROP_SCHEMA, null),
+        properties.getProperty(PROP_USER, null),
+        properties.getProperty(PROP_PASSWORD, null),
+        properties.getProperty(PROP_ADDITIONALS, null));
+  }
+
+}