From 4346216ffbed9a9adef2b90589436de543167c4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Carlos=20Rodrigo=20Mu=C3=B1oz?= Date: Mon, 7 May 2018 23:00:44 -0500 Subject: [PATCH 1/8] create template gora-ignite module --- gora-ignite/pom.xml | 177 +++ gora-ignite/src/examples/java/.gitignore | 15 + .../gora/ignite/encoders/BinaryEncoder.java | 191 +++ .../apache/gora/ignite/encoders/Encoder.java | 72 ++ .../gora/ignite/encoders/HexEncoder.java | 204 ++++ .../ignite/encoders/SignedBinaryEncoder.java | 110 ++ .../apache/gora/ignite/encoders/Utils.java | 91 ++ .../gora/ignite/encoders/package-info.java | 20 + .../org/apache/gora/ignite/package-info.java | 20 + .../apache/gora/ignite/query/IgniteQuery.java | 45 + .../gora/ignite/query/IgniteResult.java | 101 ++ .../gora/ignite/query/package-info.java | 21 + .../gora/ignite/store/IgniteMapping.java | 44 + .../apache/gora/ignite/store/IgniteStore.java | 1034 +++++++++++++++++ .../gora/ignite/store/package-info.java | 20 + .../util/FixedByteArrayOutputStream.java | 45 + .../apache/gora/ignite/util/package-info.java | 20 + .../gora/ignite/GoraIgniteTestDriver.java | 73 ++ .../org/apache/gora/ignite/package-info.java | 21 + .../ignite/store/AuthenticationTokenTest.java | 90 ++ .../gora/ignite/store/IgniteStoreTest.java | 87 ++ .../gora/ignite/store/PartitionTest.java | 96 ++ .../gora/ignite/store/package-info.java | 21 + .../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.properties | 21 + pom.xml | 1 + 29 files changed, 2942 insertions(+) create mode 100644 gora-ignite/pom.xml create mode 100644 gora-ignite/src/examples/java/.gitignore create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/package-info.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java create mode 100644 gora-ignite/src/test/resources/gora-accumulo-mapping.xml create mode 100644 gora-ignite/src/test/resources/gora.properties diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml new file mode 100644 index 000000000..ddd3a4a77 --- /dev/null +++ b/gora-ignite/pom.xml @@ -0,0 +1,177 @@ + + + + 4.0.0 + + + org.apache.gora + gora + 0.9-SNAPSHOT + ../ + + gora-ignite + bundle + + Apache Gora :: Ignite + http://gora.apache.org + The Apache Gora open source framework provides an in-memory data model and + persistence for big data. Gora supports persisting to column stores, key value stores, + document stores and RDBMSs, and analyzing the data with extensive Apache Hadoop MapReduce + support. + 2010 + + The Apache Software Foundation + http://www.apache.org/ + + + JIRA + https://issues.apache.org/jira/browse/GORA + + + Jenkins + https://builds.apache.org/job/Gora-trunk/ + + + + 1.7.1 + * + org.apache.gora.ignite*;version="${project.version}";-noimport:=true + + + + target + target/classes + ${project.artifactId}-${project.version} + target/test-classes + src/test/java + src/main/java + + + ${project.basedir}/src/test/resources + + **/* + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + ${build-helper-maven-plugin.version} + + + generate-sources + + add-source + + + + src/examples/java + + + + + + + + + + + + org.apache.gora + gora-core + + + + org.apache.gora + gora-core + test-jar + test + + + + + org.apache.ignite + ignite-core + ${ignite.version} + + + + org.apache.ignite + ignite-minicluster + ${ignite.version} + test + + + + commons-io + commons-io + + + + + jline + jline + 2.11 + test + + + + org.apache.zookeeper + zookeeper + test + + + + org.apache.avro + avro + + + + + org.slf4j + slf4j-log4j12 + + + log4j + log4j + + + javax.jms + jms + + + + + + + junit + junit + + + + org.apache.hadoop + hadoop-minicluster + + + + + diff --git a/gora-ignite/src/examples/java/.gitignore b/gora-ignite/src/examples/java/.gitignore new file mode 100644 index 000000000..09697dce6 --- /dev/null +++ b/gora-ignite/src/examples/java/.gitignore @@ -0,0 +1,15 @@ +# 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. + 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 new file mode 100644 index 000000000..ce6eded29 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java @@ -0,0 +1,191 @@ +/* + * 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); + } +} 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 new file mode 100644 index 000000000..deea4a756 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java @@ -0,0 +1,72 @@ +/** + * 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); + +} 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 new file mode 100644 index 000000000..8568ba982 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java @@ -0,0 +1,204 @@ +/** + * 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))); + } + +} 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 new file mode 100644 index 000000000..a8216f4bf --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java @@ -0,0 +1,110 @@ +/** + * 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); + } + +} 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 new file mode 100644 index 000000000..8a5980c25 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java @@ -0,0 +1,91 @@ +/** + * 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); + } + } +} 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 new file mode 100644 index 000000000..574aa24af --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/encoders/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 store related util classes for encoder. + */ +package org.apache.gora.ignite.encoders; \ No newline at end of file diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java new file mode 100644 index 000000000..a7fa7aba0 --- /dev/null +++ b/gora-ignite/src/main/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 related all classes. + */ +package org.apache.gora.ignite; \ No newline at end of file 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 new file mode 100644 index 000000000..85a59c9b7 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java @@ -0,0 +1,45 @@ +/** + * 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 extends QueryBase { + + /** + * Constructor for the query + */ + public IgniteQuery() { + super(null); + } + + /** + * Constructor for the query + * + * @param dataStore Data store used + * + */ + public IgniteQuery(DataStore dataStore) { + super(dataStore); + } + +} 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 new file mode 100644 index 000000000..416e6503b --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java @@ -0,0 +1,101 @@ +/** + * 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 extends ResultBase { + + private RowIterator iterator; + + /** + * Gets the data store used + */ + public IgniteStore getDataStore() { + return (IgniteStore) super.getDataStore(); + } + + /** + * @param dataStore + * @param query + * @param scanner + */ + public IgniteResult(DataStore dataStore, Query 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> nextRow = iterator.next(); + ByteSequence row = getDataStore().populate(nextRow, persistent); + key = ((IgniteStore) dataStore).fromBytes(getKeyClass(), row.toArray()); + + return true; + } + + @Override + public int size() { + return (int) this.limit; + } +} 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 new file mode 100644 index 000000000..b1a306c5e --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java @@ -0,0 +1,21 @@ +/** + * 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 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 new file mode 100644 index 000000000..b46c06300 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMapping.java @@ -0,0 +1,44 @@ +/** + * 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 java.util.HashMap; +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> fieldMap = new HashMap<>(); + + /** + * Look up the column associated to the Avro field. + */ + Map,String> columnMap = new HashMap<>(); + + Map tableConfig = new HashMap<>(); + String tableName; + String encoder; + +} 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 new file mode 100644 index 000000000..2f5faf9e6 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteStore.java @@ -0,0 +1,1034 @@ +/** + * 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 java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Properties; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; + +import org.apache.ignite.core.client.IgniteException; +import org.apache.ignite.core.client.IgniteSecurityException; +import org.apache.ignite.core.client.BatchWriter; +import org.apache.ignite.core.client.BatchWriterConfig; +import org.apache.ignite.core.client.Connector; +import org.apache.ignite.core.client.IsolatedScanner; +import org.apache.ignite.core.client.IteratorSetting; +import org.apache.ignite.core.client.MutationsRejectedException; +import org.apache.ignite.core.client.RowIterator; +import org.apache.ignite.core.client.Scanner; +import org.apache.ignite.core.client.TableDeletedException; +import org.apache.ignite.core.client.TableExistsException; +import org.apache.ignite.core.client.TableNotFoundException; +import org.apache.ignite.core.client.TableOfflineException; +import org.apache.ignite.core.client.ZooKeeperInstance; +import org.apache.ignite.core.client.impl.ClientContext; +import org.apache.ignite.core.client.impl.Tables; +import org.apache.ignite.core.client.impl.TabletLocator; +import org.apache.ignite.core.client.mock.MockConnector; +import org.apache.ignite.core.client.mock.MockInstance; +import org.apache.ignite.core.client.mock.impl.MockTabletLocator; +import org.apache.ignite.core.client.security.tokens.AuthenticationToken; +import org.apache.ignite.core.client.security.tokens.PasswordToken; +import org.apache.ignite.core.conf.IgniteConfiguration; +import org.apache.ignite.core.data.ByteSequence; +import org.apache.ignite.core.data.Key; +import org.apache.ignite.core.data.impl.KeyExtent; +import org.apache.ignite.core.data.Mutation; +import org.apache.ignite.core.data.Range; +import org.apache.ignite.core.data.Value; +import org.apache.ignite.core.iterators.SortedKeyIterator; +import org.apache.ignite.core.iterators.user.TimestampFilter; +import org.apache.ignite.core.master.state.tables.TableState; +import org.apache.ignite.core.security.Authorizations; +import org.apache.ignite.core.security.ColumnVisibility; +import org.apache.ignite.core.client.impl.Credentials; +import org.apache.ignite.core.util.Pair; +import org.apache.ignite.core.util.UtilWaitThread; +import org.apache.avro.Schema; +import org.apache.avro.Schema.Field; +import org.apache.avro.Schema.Type; +import org.apache.avro.generic.GenericData; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.Decoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.io.EncoderFactory; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.util.Utf8; +import org.apache.gora.ignite.encoders.BinaryEncoder; +import org.apache.gora.ignite.encoders.Encoder; +import org.apache.gora.ignite.query.IgniteQuery; +import org.apache.gora.ignite.query.IgniteResult; +import org.apache.gora.persistency.impl.DirtyListWrapper; +import org.apache.gora.persistency.impl.DirtyMapWrapper; +import org.apache.gora.persistency.impl.PersistentBase; +import org.apache.gora.query.PartitionQuery; +import org.apache.gora.query.Query; +import org.apache.gora.query.Result; +import org.apache.gora.query.impl.PartitionQueryImpl; +import org.apache.gora.store.DataStoreFactory; +import org.apache.gora.store.impl.DataStoreBase; +import org.apache.gora.util.AvroUtils; +import org.apache.gora.util.GoraException; +import org.apache.gora.util.IOUtils; +import org.apache.hadoop.io.Text; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.w3c.dom.Document; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; + +/** + * Implementation of a Ignite data store to be used by gora. + * + * @param + * class to be used for the key + * @param + * class to be persisted within the store + */ +public class IgniteStore extends DataStoreBase { + + protected static final String MOCK_PROPERTY = "ignite.mock"; + protected static final String INSTANCE_NAME_PROPERTY = "ignite.instance"; + protected static final String ZOOKEEPERS_NAME_PROPERTY = "ignite.zookeepers"; + protected static final String USERNAME_PROPERTY = "ignite.user"; + protected static final String PASSWORD_PROPERTY = "ignite.password"; + protected static final String DEFAULT_MAPPING_FILE = "gora-ignite-mapping.xml"; + + private final static String UNKOWN = "Unknown type "; + + private Connector conn; + private BatchWriter batchWriter; + private IgniteMapping mapping; + private Credentials credentials; + private Encoder encoder; + + public static final Logger LOG = LoggerFactory.getLogger(IgniteStore.class); + + public Object fromBytes(Schema schema, byte[] data) throws IOException { + Schema fromSchema = null; + if (schema.getType() == Type.UNION) { + try { + Decoder decoder = DecoderFactory.get().binaryDecoder(data, null); + int unionIndex = decoder.readIndex(); + List possibleTypes = schema.getTypes(); + fromSchema = possibleTypes.get(unionIndex); + Schema effectiveSchema = possibleTypes.get(unionIndex); + if (effectiveSchema.getType() == Type.NULL) { + decoder.readNull(); + return null; + } else { + data = decoder.readBytes(null).array(); + } + } catch (IOException e) { + LOG.error(e.getMessage()); + throw new GoraException("Error decoding union type: ", e); + } + } else { + fromSchema = schema; + } + return fromBytes(encoder, fromSchema, data); + } + + public static Object fromBytes(Encoder encoder, Schema schema, byte data[]) throws IOException { + switch (schema.getType()) { + case BOOLEAN: + return encoder.decodeBoolean(data); + case DOUBLE: + return encoder.decodeDouble(data); + case FLOAT: + return encoder.decodeFloat(data); + case INT: + return encoder.decodeInt(data); + case LONG: + return encoder.decodeLong(data); + case STRING: + return new Utf8(data); + case BYTES: + return ByteBuffer.wrap(data); + case ENUM: + return AvroUtils.getEnumValue(schema, encoder.decodeInt(data)); + case ARRAY: + break; + case FIXED: + break; + case MAP: + break; + case NULL: + break; + case RECORD: + break; + case UNION: + break; + default: + break; + } + throw new IllegalArgumentException(UNKOWN + schema.getType()); + + } + + private static byte[] getBytes(Text text) { + byte[] bytes = text.getBytes(); + if (bytes.length != text.getLength()) { + bytes = new byte[text.getLength()]; + System.arraycopy(text.getBytes(), 0, bytes, 0, bytes.length); + } + return bytes; + } + + public K fromBytes(Class clazz, byte[] val) { + return fromBytes(encoder, clazz, val); + } + + @SuppressWarnings("unchecked") + public static K fromBytes(Encoder encoder, Class clazz, byte[] val) { + try { + if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { + return (K) Byte.valueOf(encoder.decodeByte(val)); + } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { + return (K) Boolean.valueOf(encoder.decodeBoolean(val)); + } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { + return (K) Short.valueOf(encoder.decodeShort(val)); + } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { + return (K) Integer.valueOf(encoder.decodeInt(val)); + } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { + return (K) Long.valueOf(encoder.decodeLong(val)); + } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { + return (K) Float.valueOf(encoder.decodeFloat(val)); + } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { + return (K) Double.valueOf(encoder.decodeDouble(val)); + } else if (clazz.equals(String.class)) { + return (K) new String(val, "UTF-8"); + } else if (clazz.equals(Utf8.class)) { + return (K) new Utf8(val); + } + + throw new IllegalArgumentException(UNKOWN + clazz.getName()); + } catch (IOException ioe) { + LOG.error(ioe.getMessage()); + throw new RuntimeException(ioe); + } + } + + private static byte[] copyIfNeeded(byte b[], int offset, int len) { + if (len != b.length || offset != 0) { + byte[] copy = new byte[len]; + System.arraycopy(b, offset, copy, 0, copy.length); + b = copy; + } + return b; + } + + public byte[] toBytes(Schema toSchema, Object o) { + if (toSchema != null && toSchema.getType() == Type.UNION) { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + org.apache.avro.io.BinaryEncoder avroEncoder = EncoderFactory.get().binaryEncoder(baos, null); + int unionIndex = 0; + try { + if (o == null) { + unionIndex = firstNullSchemaTypeIndex(toSchema); + avroEncoder.writeIndex(unionIndex); + avroEncoder.writeNull(); + } else { + unionIndex = firstNotNullSchemaTypeIndex(toSchema); + avroEncoder.writeIndex(unionIndex); + avroEncoder.writeBytes(toBytes(o)); + } + avroEncoder.flush(); + return baos.toByteArray(); + } catch (IOException e) { + LOG.error(e.getMessage()); + return toBytes(o); + } + } else { + return toBytes(o); + } + } + + private int firstNullSchemaTypeIndex(Schema toSchema) { + List possibleTypes = toSchema.getTypes(); + int unionIndex = 0; + for (int i = 0; i < possibleTypes.size(); i++ ) { + Type pType = possibleTypes.get(i).getType(); + if (pType == Type.NULL) { // FIXME HUGE kludge to pass tests + unionIndex = i; break; + } + } + return unionIndex; + } + + private int firstNotNullSchemaTypeIndex(Schema toSchema) { + List possibleTypes = toSchema.getTypes(); + int unionIndex = 0; + for (int i = 0; i < possibleTypes.size(); i++ ) { + Type pType = possibleTypes.get(i).getType(); + if (pType != Type.NULL) { // FIXME HUGE kludge to pass tests + unionIndex = i; break; + } + } + return unionIndex; + } + + public byte[] toBytes(Object o) { + return toBytes(encoder, o); + } + + public static byte[] toBytes(Encoder encoder, Object o) { + + try { + if (o instanceof String) { + return ((String) o).getBytes("UTF-8"); + } else if (o instanceof Utf8) { + return copyIfNeeded(((Utf8) o).getBytes(), 0, ((Utf8) o).getByteLength()); + } else if (o instanceof ByteBuffer) { + return copyIfNeeded(((ByteBuffer) o).array(), ((ByteBuffer) o).arrayOffset() + ((ByteBuffer) o).position(), ((ByteBuffer) o).remaining()); + } else if (o instanceof Long) { + return encoder.encodeLong((Long) o); + } else if (o instanceof Integer) { + return encoder.encodeInt((Integer) o); + } else if (o instanceof Short) { + return encoder.encodeShort((Short) o); + } else if (o instanceof Byte) { + return encoder.encodeByte((Byte) o); + } else if (o instanceof Boolean) { + return encoder.encodeBoolean((Boolean) o); + } else if (o instanceof Float) { + return encoder.encodeFloat((Float) o); + } else if (o instanceof Double) { + return encoder.encodeDouble((Double) o); + } else if (o instanceof Enum) { + return encoder.encodeInt(((Enum) o).ordinal()); + } + } catch (IOException ioe) { + throw new RuntimeException(ioe); + } + + throw new IllegalArgumentException(UNKOWN + o.getClass().getName()); + } + + private BatchWriter getBatchWriter() throws IOException { + if (batchWriter == null) + try { + BatchWriterConfig batchWriterConfig = new BatchWriterConfig(); + batchWriterConfig.setMaxMemory(10000000); + batchWriterConfig.setMaxLatency(60000L, TimeUnit.MILLISECONDS); + batchWriterConfig.setMaxWriteThreads(4); + batchWriter = conn.createBatchWriter(mapping.tableName, batchWriterConfig); + } catch (TableNotFoundException e) { + throw new IOException(e); + } + return batchWriter; + } + + /** + * Initialize the data store by reading the credentials, setting the client's properties up and + * reading the mapping file. Initialize is called when then the call to + * {@link org.apache.gora.store.DataStoreFactory#createDataStore} is made. + * + * @param keyClass + * @param persistentClass + * @param properties + */ + @Override + public void initialize(Class keyClass, Class persistentClass, Properties properties) throws GoraException { + super.initialize(keyClass, persistentClass, properties); + + try { + + String mock = DataStoreFactory.findProperty(properties, this, MOCK_PROPERTY, null); + String mappingFile = DataStoreFactory.getMappingFile(properties, this, DEFAULT_MAPPING_FILE); + String user = DataStoreFactory.findProperty(properties, this, USERNAME_PROPERTY, null); + String password = DataStoreFactory.findProperty(properties, this, PASSWORD_PROPERTY, null); + + mapping = readMapping(mappingFile); + + if (mapping.encoder == null || "".equals(mapping.encoder)) { + encoder = new BinaryEncoder(); + } else { + encoder = (Encoder) getClass().getClassLoader().loadClass(mapping.encoder).newInstance(); + } + + AuthenticationToken token = new PasswordToken(password); + if (mock == null || !mock.equals("true")) { + String instance = DataStoreFactory.findProperty(properties, this, INSTANCE_NAME_PROPERTY, null); + String zookeepers = DataStoreFactory.findProperty(properties, this, ZOOKEEPERS_NAME_PROPERTY, null); + conn = new ZooKeeperInstance(instance, zookeepers).getConnector(user, token); + } else { + conn = new MockInstance().getConnector(user, token); + } + credentials = new Credentials(user, token); + + if (autoCreateSchema && !schemaExists()) + createSchema(); + + } catch (IOException | InstantiationException | IllegalAccessException | + ClassNotFoundException | IgniteException | IgniteSecurityException e) { + throw new GoraException(e); + } + } + + protected IgniteMapping readMapping(String filename) throws IOException { + try { + + IgniteMapping mapping = new IgniteMapping(); + + DocumentBuilder db = DocumentBuilderFactory.newInstance().newDocumentBuilder(); + Document dom = db.parse(getClass().getClassLoader().getResourceAsStream(filename)); + + Element root = dom.getDocumentElement(); + + NodeList nl = root.getElementsByTagName("class"); + for (int i = 0; i < nl.getLength(); i++) { + + Element classElement = (Element) nl.item(i); + if (classElement.getAttribute("keyClass").equals(keyClass.getCanonicalName()) + && classElement.getAttribute("name").equals(persistentClass.getCanonicalName())) { + + mapping.tableName = getSchemaName(classElement.getAttribute("table"), persistentClass); + mapping.encoder = classElement.getAttribute("encoder"); + + NodeList fields = classElement.getElementsByTagName("field"); + for (int j = 0; j < fields.getLength(); j++) { + Element fieldElement = (Element) fields.item(j); + + String name = fieldElement.getAttribute("name"); + String family = fieldElement.getAttribute("family"); + String qualifier = fieldElement.getAttribute("qualifier"); + if ("".equals(qualifier)) + qualifier = null; + + Pair col = new Pair<>(new Text(family), qualifier == null ? null : new Text(qualifier)); + mapping.fieldMap.put(name, col); + mapping.columnMap.put(col, name); + } + } + + } + + if (mapping.tableName == null) { + throw new GoraException("Please define the ignite 'table' name mapping in " + filename + " for " + persistentClass.getCanonicalName()); + } + + nl = root.getElementsByTagName("table"); + for (int i = 0; i < nl.getLength(); i++) { + Element tableElement = (Element) nl.item(i); + if (tableElement.getAttribute("name").equals(mapping.tableName)) { + NodeList configs = tableElement.getElementsByTagName("config"); + for (int j = 0; j < configs.getLength(); j++) { + Element configElement = (Element) configs.item(j); + String key = configElement.getAttribute("key"); + String val = configElement.getAttribute("value"); + mapping.tableConfig.put(key, val); + } + } + } + + return mapping; + } catch (Exception ex) { + throw new IOException("Unable to read " + filename, ex); + } + + } + + @Override + public String getSchemaName() { + return mapping.tableName; + } + + @Override + public void createSchema() throws GoraException { + try { + conn.tableOperations().create(mapping.tableName); + Set> es = mapping.tableConfig.entrySet(); + for (Entry entry : es) { + conn.tableOperations().setProperty(mapping.tableName, entry.getKey(), entry.getValue()); + } + + } catch (TableExistsException e) { + LOG.debug(e.getMessage(), e); + // Assume this is not an error + } catch (IgniteException | IgniteSecurityException e) { + throw new GoraException(e); + } + } + + @Override + public void deleteSchema() throws GoraException { + try { + if (batchWriter != null) + batchWriter.close(); + batchWriter = null; + conn.tableOperations().delete(mapping.tableName); + } catch (TableNotFoundException e) { + // Ignore. Delete a non existant schema is a success + } catch (IgniteException | IgniteSecurityException e) { + throw new GoraException(e); + } + } + + @Override + public boolean schemaExists() throws GoraException { + try { + return conn.tableOperations().exists(mapping.tableName); + } catch (Exception e) { + throw new GoraException(e); + } + } + + public ByteSequence populate(Iterator> iter, T persistent) throws IOException { + ByteSequence row = null; + + Map currentMap = null; + List currentArray = null; + Text currentFam = null; + int currentPos = 0; + Schema currentSchema = null; + Field currentField = null; + + BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(new byte[0], null); + + while (iter.hasNext()) { + Entry entry = iter.next(); + + if (row == null) { + row = entry.getKey().getRowData(); + } + byte[] val = entry.getValue().get(); + + Field field = fieldMap.get(getFieldName(entry)); + + if (currentMap != null) { + if (currentFam.equals(entry.getKey().getColumnFamily())) { + currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), + fromBytes(currentSchema, entry.getValue().get())); + continue; + } else { + persistent.put(currentPos, currentMap); + currentMap = null; + } + } else if (currentArray != null) { + if (currentFam.equals(entry.getKey().getColumnFamily())) { + currentArray.add(fromBytes(currentSchema, entry.getValue().get())); + continue; + } else { + persistent.put(currentPos, new GenericData.Array(currentField.schema(), currentArray)); + currentArray = null; + } + } + + switch (field.schema().getType()) { + case MAP: // first entry only. Next are handled above on the next loop + currentMap = new DirtyMapWrapper<>(new HashMap()); + currentPos = field.pos(); + currentFam = entry.getKey().getColumnFamily(); + currentSchema = field.schema().getValueType(); + + currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), + fromBytes(currentSchema, entry.getValue().get())); + break; + case ARRAY: + currentArray = new DirtyListWrapper<>(new ArrayList<>()); + currentPos = field.pos(); + currentFam = entry.getKey().getColumnFamily(); + currentSchema = field.schema().getElementType(); + currentField = field; + + currentArray.add(fromBytes(currentSchema, entry.getValue().get())); + + break; + case UNION:// default value of null acts like union with null + Schema effectiveSchema = field.schema().getTypes() + .get(firstNotNullSchemaTypeIndex(field.schema())); + // map and array were coded without union index so need to be read the same way + if (effectiveSchema.getType() == Type.ARRAY) { + currentArray = new DirtyListWrapper<>(new ArrayList<>()); + currentPos = field.pos(); + currentFam = entry.getKey().getColumnFamily(); + currentSchema = field.schema().getElementType(); + currentField = field; + + currentArray.add(fromBytes(currentSchema, entry.getValue().get())); + break; + } + else if (effectiveSchema.getType() == Type.MAP) { + currentMap = new DirtyMapWrapper<>(new HashMap()); + currentPos = field.pos(); + currentFam = entry.getKey().getColumnFamily(); + currentSchema = effectiveSchema.getValueType(); + + currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), + fromBytes(currentSchema, entry.getValue().get())); + break; + } + // continue like a regular top-level union + case RECORD: + SpecificDatumReader reader = new SpecificDatumReader(field.schema()); + persistent.put(field.pos(), reader.read(null, DecoderFactory.get().binaryDecoder(val, decoder))); + break; + default: + persistent.put(field.pos(), fromBytes(field.schema(), entry.getValue().get())); + } + } + + if (currentMap != null) { + persistent.put(currentPos, currentMap); + } else if (currentArray != null) { + persistent.put(currentPos, new GenericData.Array(currentField.schema(), currentArray)); + } + + persistent.clearDirty(); + + return row; + } + + /** + * Retrieve field name from entry. + * @param entry The Key-Value entry + * @return String The field name + */ + private String getFieldName(Entry entry) { + String fieldName = mapping.columnMap.get(new Pair<>(entry.getKey().getColumnFamily(), + entry.getKey().getColumnQualifier())); + if (fieldName == null) { + fieldName = mapping.columnMap.get(new Pair(entry.getKey().getColumnFamily(), null)); + } + return fieldName; + } + + private void setFetchColumns(Scanner scanner, String[] fields) { + fields = getFieldsToQuery(fields); + for (String field : fields) { + Pair col = mapping.fieldMap.get(field); + if (col != null) { + if (col.getSecond() == null) { + scanner.fetchColumnFamily(col.getFirst()); + } else { + scanner.fetchColumn(col.getFirst(), col.getSecond()); + } + } else { + LOG.error("Mapping not found for field: {}", field); + } + } + } + + @Override + public T get(K key, String[] fields) throws GoraException { + try { + // TODO make isolated scanner optional? + Scanner scanner = new IsolatedScanner(conn.createScanner(mapping.tableName, Authorizations.EMPTY)); + Range rowRange = new Range(new Text(toBytes(key))); + + scanner.setRange(rowRange); + setFetchColumns(scanner, fields); + + T persistent = newPersistent(); + ByteSequence row = populate(scanner.iterator(), persistent); + if (row == null) + return null; + return persistent; + } catch (Exception e) { + throw new GoraException(e); + } + } + + @Override + public void put(K key, T val) throws GoraException { + + try{ + Mutation m = new Mutation(new Text(toBytes(key))); + + Schema schema = val.getSchema(); + List fields = schema.getFields(); + int count = 0; + + for (int i = 0; i < fields.size(); i++) { + if (!val.isDirty(i)) { + continue; + } + Field field = fields.get(i); + + Object o = val.get(field.pos()); + + Pair col = mapping.fieldMap.get(field.name()); + + if (col == null) { + throw new GoraException("Please define the gora to ignite mapping for field " + field.name()); + } + + switch (field.schema().getType()) { + case MAP: + count = putMap(m, count, field.schema().getValueType(), o, col, field.name()); + break; + case ARRAY: + count = putArray(m, count, o, col, field.name()); + break; + case UNION: // default value of null acts like union with null + Schema effectiveSchema = field.schema().getTypes() + .get(firstNotNullSchemaTypeIndex(field.schema())); + // map and array need to compute qualifier + if (effectiveSchema.getType() == Type.ARRAY) { + count = putArray(m, count, o, col, field.name()); + break; + } + else if (effectiveSchema.getType() == Type.MAP) { + count = putMap(m, count, effectiveSchema.getValueType(), o, col, field.name()); + break; + } + // continue like a regular top-level union + case RECORD: + final SpecificDatumWriter writer = new SpecificDatumWriter<>(field.schema()); + final byte[] byteData = IOUtils.serialize(writer,o); + m.put(col.getFirst(), col.getSecond(), new Value(byteData)); + count++; + break; + default: + m.put(col.getFirst(), col.getSecond(), new Value(toBytes(o))); + count++; + } + + } + + if (count > 0) + try { + getBatchWriter().addMutation(m); + } catch (MutationsRejectedException e) { + LOG.error(e.getMessage(), e); + } + } catch (GoraException e) { + throw e; + } catch (Exception e) { + throw new GoraException(e); + } + } + + private int putMap(Mutation m, int count, Schema valueType, Object o, Pair col, String fieldName) throws GoraException { + + // First of all we delete map field on ignite store + Text rowKey = new Text(m.getRow()); + Query query = newQuery(); + query.setFields(fieldName); + query.setStartKey((K)rowKey.toString()); + query.setEndKey((K)rowKey.toString()); + deleteByQuery(query); + flush(); + if (o == null){ + return 0; + } + + Set es = ((Map)o).entrySet(); + for (Object entry : es) { + Object mapKey = ((Entry) entry).getKey(); + Object mapVal = ((Entry) entry).getValue(); + if ((o instanceof DirtyMapWrapper && ((DirtyMapWrapper)o).isDirty()) + || !(o instanceof DirtyMapWrapper)) { + m.put(col.getFirst(), new Text(toBytes(mapKey)), new Value(toBytes(valueType, mapVal))); + count++; + } + // TODO map value deletion + } + return count; + } + + private int putArray(Mutation m, int count, Object o, Pair col, String fieldName) throws GoraException { + + // First of all we delete array field on ignite store + Text rowKey = new Text(m.getRow()); + Query query = newQuery(); + query.setFields(fieldName); + query.setStartKey((K)rowKey.toString()); + query.setEndKey((K)rowKey.toString()); + deleteByQuery(query); + flush(); + if (o == null){ + return 0; + } + + List array = (List) o; // both GenericArray and DirtyListWrapper + int j = 0; + for (Object item : array) { + m.put(col.getFirst(), new Text(toBytes(j++)), new Value(toBytes(item))); + count++; + } + return count; + } + + @Override + public boolean delete(K key) throws GoraException { + Query q = newQuery(); + q.setKey(key); + return deleteByQuery(q) > 0; + } + + @Override + public long deleteByQuery(Query query) throws GoraException { + try { + Scanner scanner = createScanner(query); + // add iterator that drops values on the server side + scanner.addScanIterator(new IteratorSetting(Integer.MAX_VALUE, SortedKeyIterator.class)); + RowIterator iterator = new RowIterator(scanner.iterator()); + + long count = 0; + + while (iterator.hasNext()) { + Iterator> row = iterator.next(); + Mutation m = null; + while (row.hasNext()) { + Entry entry = row.next(); + Key key = entry.getKey(); + if (m == null) + m = new Mutation(key.getRow()); + // TODO optimize to avoid continually creating column vis? prob does not matter for empty + m.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibility()), key.getTimestamp()); + } + getBatchWriter().addMutation(m); + count++; + } + + return count; + } catch (Exception e) { + throw new GoraException(e); + } + } + + private Range createRange(Query query) { + Text startRow = null; + Text endRow = null; + + if (query.getStartKey() != null) + startRow = new Text(toBytes(query.getStartKey())); + + if (query.getEndKey() != null) + endRow = new Text(toBytes(query.getEndKey())); + + return new Range(startRow, true, endRow, true); + + } + + private Scanner createScanner(Query query) throws TableNotFoundException { + // TODO make isolated scanner optional? + Scanner scanner = new IsolatedScanner(conn.createScanner(mapping.tableName, Authorizations.EMPTY)); + setFetchColumns(scanner, query.getFields()); + + scanner.setRange(createRange(query)); + + if (query.getStartTime() != -1 || query.getEndTime() != -1) { + IteratorSetting is = new IteratorSetting(30, TimestampFilter.class); + if (query.getStartTime() != -1) + TimestampFilter.setStart(is, query.getStartTime(), true); + if (query.getEndTime() != -1) + TimestampFilter.setEnd(is, query.getEndTime(), true); + + scanner.addScanIterator(is); + } + + return scanner; + } + + /** + * Execute the query and return the result. + */ + @Override + public Result execute(Query query) throws GoraException { + try { + Scanner scanner = createScanner(query); + return new IgniteResult<>(this, query, scanner); + } catch (TableNotFoundException e) { + throw new GoraException(e) ; + } + } + + @Override + public Query newQuery() { + return new IgniteQuery<>(this); + } + + Text pad(Text key, int bytes) { + if (key.getLength() < bytes) + key = new Text(key); + + while (key.getLength() < bytes) { + key.append(new byte[] {0}, 0, 1); + } + + return key; + } + + @Override + public List> getPartitions(Query query) throws GoraException { + try { + TabletLocator tl; + if (conn instanceof MockConnector) + tl = new MockTabletLocator(); + else + tl = TabletLocator.getLocator(new ClientContext(conn.getInstance(), credentials, IgniteConfiguration.getTableConfiguration(conn, Tables.getTableId(conn.getInstance(), mapping.tableName))), new Text(Tables.getTableId(conn.getInstance(), mapping.tableName))); + + Map>> binnedRanges = new HashMap<>(); + + tl.invalidateCache(); + while (tl.binRanges(new ClientContext(conn.getInstance(), credentials, IgniteConfiguration.getTableConfiguration(conn, Tables.getTableId(conn.getInstance(), mapping.tableName))), Collections.singletonList(createRange(query)), binnedRanges).size() > 0) { + // TODO log? + if (!Tables.exists(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName))) + throw new TableDeletedException(Tables.getTableId(conn.getInstance(), mapping.tableName)); + else if (Tables.getTableState(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName)) == TableState.OFFLINE) + throw new TableOfflineException(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName)); + UtilWaitThread.sleep(100); + tl.invalidateCache(); + } + + List> ret = new ArrayList<>(); + + Text startRow = null; + Text endRow = null; + if (query.getStartKey() != null) + startRow = new Text(toBytes(query.getStartKey())); + if (query.getEndKey() != null) + endRow = new Text(toBytes(query.getEndKey())); + + //hadoop expects hostnames, ignite keeps track of IPs... so need to convert + HashMap hostNameCache = new HashMap<>(); + + for (Entry>> entry : binnedRanges.entrySet()) { + String ip = entry.getKey().split(":", 2)[0]; + String location = hostNameCache.get(ip); + if (location == null) { + InetAddress inetAddress = InetAddress.getByName(ip); + location = inetAddress.getHostName(); + hostNameCache.put(ip, location); + } + + Map> tablets = entry.getValue(); + for (KeyExtent ke : tablets.keySet()) { + + K startKey = null; + if (startRow == null || !ke.contains(startRow)) { + if (ke.getPrevEndRow() != null) { + startKey = followingKey(encoder, getKeyClass(), getBytes(ke.getPrevEndRow())); + } + } else { + startKey = fromBytes(getKeyClass(), getBytes(startRow)); + } + + K endKey = null; + if (endRow == null || !ke.contains(endRow)) { + if (ke.getEndRow() != null) + endKey = lastPossibleKey(encoder, getKeyClass(), getBytes(ke.getEndRow())); + } else { + endKey = fromBytes(getKeyClass(), getBytes(endRow)); + } + + PartitionQueryImpl pqi = new PartitionQueryImpl<>(query, startKey, endKey, location); + pqi.setConf(getConf()); + ret.add(pqi); + } + } + + return ret; + } catch (Exception e) { + throw new GoraException(e); + } + + } + + static K lastPossibleKey(Encoder encoder, Class clazz, byte[] er) { + + if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { + throw new UnsupportedOperationException(); + } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { + throw new UnsupportedOperationException(); + } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { + return fromBytes(encoder, clazz, encoder.lastPossibleKey(2, er)); + } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { + return fromBytes(encoder, clazz, encoder.lastPossibleKey(4, er)); + } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { + return fromBytes(encoder, clazz, encoder.lastPossibleKey(8, er)); + } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { + return fromBytes(encoder, clazz, encoder.lastPossibleKey(4, er)); + } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { + return fromBytes(encoder, clazz, encoder.lastPossibleKey(8, er)); + } else if (clazz.equals(String.class)) { + throw new UnsupportedOperationException(); + } else if (clazz.equals(Utf8.class)) { + return fromBytes(encoder, clazz, er); + } + + throw new IllegalArgumentException(UNKOWN + clazz.getName()); + } + + @SuppressWarnings("unchecked") + static K followingKey(Encoder encoder, Class clazz, byte[] per) { + + if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { + return (K) Byte.valueOf(encoder.followingKey(1, per)[0]); + } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { + throw new UnsupportedOperationException(); + } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { + return fromBytes(encoder, clazz, encoder.followingKey(2, per)); + } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { + return fromBytes(encoder, clazz, encoder.followingKey(4, per)); + } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { + return fromBytes(encoder, clazz, encoder.followingKey(8, per)); + } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { + return fromBytes(encoder, clazz, encoder.followingKey(4, per)); + } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { + return fromBytes(encoder, clazz, encoder.followingKey(8, per)); + } else if (clazz.equals(String.class)) { + throw new UnsupportedOperationException(); + } else if (clazz.equals(Utf8.class)) { + return fromBytes(encoder, clazz, Arrays.copyOf(per, per.length + 1)); + } + + throw new IllegalArgumentException(UNKOWN + clazz.getName()); + } + + @Override + public void flush() throws GoraException { + try { + if (batchWriter != null) { + batchWriter.flush(); + } + } catch (Exception e) { + throw new GoraException(e); + } + } + + @Override + public void close() { + try { + if (batchWriter != null) { + batchWriter.close(); + batchWriter = null; + } + } catch (MutationsRejectedException e) { + LOG.error(e.getMessage(), e); + } + } +} 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 new file mode 100644 index 000000000..62cba6785 --- /dev/null +++ b/gora-ignite/src/main/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 all the Ignite store related classes. + */ +package org.apache.gora.ignite.store; \ No newline at end of file diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java b/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java new file mode 100644 index 000000000..97fb46ad1 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java @@ -0,0 +1,45 @@ +/** + * 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.util; + +import java.io.IOException; +import java.io.OutputStream; + +/** + * It is a implementation of {@link java.io.OutputStream} must always provide at least a method that writes one byte of output. + */ +public class FixedByteArrayOutputStream extends OutputStream { + + private int i; + byte out[]; + + public FixedByteArrayOutputStream(byte out[]) { + this.out = out; + } + + @Override + public void write(int b) throws IOException { + out[i++] = (byte) b; + } + + @Override + public void write(byte b[], int off, int len) throws IOException { + System.arraycopy(b, off, out, i, len); + i += len; + } + +} \ No newline at end of file diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java new file mode 100644 index 000000000..eedb84b5c --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/util/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 store related util classes. + */ +package org.apache.gora.ignite.util; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java b/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java new file mode 100644 index 000000000..1e0371731 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java @@ -0,0 +1,73 @@ +/** + * 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; + +import org.apache.ignite.minicluster.MiniIgniteCluster; +import org.apache.ignite.minicluster.MiniIgniteConfig; +import org.apache.gora.GoraTestDriver; +import org.apache.gora.ignite.store.IgniteStore; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +/** + * @author lmcgibbn + * + */ +public class GoraIgniteTestDriver extends GoraTestDriver { + + private static final Logger LOG = LoggerFactory.getLogger(GoraIgniteTestDriver.class); + private static MiniIgniteCluster cluster = null; + private static final String PASSWORD = "drowssap"; + + @Rule + public TemporaryFolder tmpDir = new TemporaryFolder(); + + public GoraIgniteTestDriver() throws Exception { + super(IgniteStore.class); + } + + @Override + public void setUpClass() throws IOException, InterruptedException { + log.info("Starting Ignite MiniIgniteCluster..."); + try { + tmpDir.create(); + MiniIgniteConfig miniCfg = new MiniIgniteConfig(tmpDir.getRoot(), PASSWORD); + miniCfg.setInstanceName("goraTest"); + miniCfg.setZooKeeperPort(56321); + cluster = new MiniIgniteCluster(miniCfg); + cluster.start(); + } catch (Exception e) { + LOG.error("Error starting Ignite MiniIgniteCluster: {}", e.getMessage()); + // cleanup + tearDownClass(); + } + } + + @Override + public void tearDownClass() throws IOException, InterruptedException { + log.info("Shutting down Ignite MiniIgniteCluster..."); + if (cluster != null) { + cluster.stop(); + } + tmpDir.delete(); + } +} 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 000000000..f536961c7 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java @@ -0,0 +1,21 @@ +/** + * 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. + */ +/** + * Tests for gora-ignite including + * the test driver for {@link org.apache.gora.ignite.store.IgniteStoreTest} + */ +package org.apache.gora.ignite; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java new file mode 100644 index 000000000..bb77beea7 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java @@ -0,0 +1,90 @@ +/** + * 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 java.util.List; +import java.util.Properties; +import org.apache.ignite.minicluster.MiniIgniteCluster; +import org.apache.gora.examples.generated.Employee; +import org.apache.gora.query.PartitionQuery; +import org.apache.gora.store.DataStore; +import org.apache.gora.store.DataStoreFactory; +import org.apache.hadoop.conf.Configuration; +import org.junit.After; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Tests authentication token is serialized correctly. + */ +public class AuthenticationTokenTest { + private static final Logger LOG = LoggerFactory.getLogger(AuthenticationTokenTest.class); + + private static final String GORA_DATASTORE = + DataStoreFactory.GORA + "." + DataStoreFactory.DATASTORE + "."; + private static final String PASSWORD = "password"; + + @Rule + public TemporaryFolder temporaryFolder = new TemporaryFolder(); + + private MiniIgniteCluster cluster; + private DataStore employeeStore; + + @Before + @SuppressWarnings("unchecked") + public void setUp() throws Exception { + cluster = new MiniIgniteCluster(temporaryFolder.getRoot(), PASSWORD); + cluster.start(); + + Properties properties = DataStoreFactory.createProps(); + properties.setProperty( + GORA_DATASTORE + IgniteStore.MOCK_PROPERTY, + "false"); + properties.setProperty( + GORA_DATASTORE + IgniteStore.INSTANCE_NAME_PROPERTY, + cluster.getInstanceName()); + properties.setProperty( + GORA_DATASTORE + IgniteStore.ZOOKEEPERS_NAME_PROPERTY, + cluster.getZooKeepers()); + properties.setProperty( + GORA_DATASTORE + IgniteStore.PASSWORD_PROPERTY, + PASSWORD); + + employeeStore = DataStoreFactory.createDataStore( + IgniteStore.class, + String.class, + Employee.class, + new Configuration(), + properties); + } + + @After + public void tearDown() throws Exception { + cluster.stop(); + } + + @Test + public void testAuthenticationTokenIsSerializedCorrectly() throws Exception { + List> partitions = + employeeStore.getPartitions(employeeStore.newQuery()); + LOG.debug("partitions {}", partitions); + } +} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java new file mode 100644 index 000000000..b42d3442d --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java @@ -0,0 +1,87 @@ +/** + * 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 java.io.IOException; + +import org.apache.gora.ignite.GoraIgniteTestDriver; +import org.apache.gora.examples.generated.Employee; +import org.apache.gora.examples.generated.WebPage; +import org.apache.gora.store.DataStore; +import org.apache.gora.store.DataStoreFactory; +import org.apache.gora.store.DataStoreTestBase; +import static org.apache.gora.store.DataStoreTestBase.log; +import org.apache.gora.store.DataStoreTestUtil; +import static org.apache.gora.store.DataStoreTestUtil.testResultSize; +import org.apache.hadoop.conf.Configuration; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; + +/** + * Tests extending {@link org.apache.gora.store.DataStoreTestBase} + * which run the base JUnit test suite for Gora. + */ +public class IgniteStoreTest extends DataStoreTestBase { + + static { + try { + setTestDriver(new GoraIgniteTestDriver()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + @Before + public void setUp() throws Exception { + super.setUp(); + } + + public GoraIgniteTestDriver getTestDriver() { + return (GoraIgniteTestDriver) testDriver; + } + + //Until GORA-66 is resolved this test will always fail, so + //do not run it + @Ignore("skipped until GORA-66 is resolved") + @Override + public void testDeleteByQueryFields() throws IOException { + } + + @Test + @Ignore("Ignite does not support Result#size() without limit set") + @Override + public void testResultSize() throws Exception { + } + + @Test + @Ignore("Ignite does not support Result#size() without limit set") + @Override + public void testResultSizeStartKey() throws Exception { + } + + @Ignore("Ignite does not support Result#size() without limit set") + @Override + public void testResultSizeEndKey() throws Exception { + } + + @Test + @Ignore("Ignite does not support Result#size() without limit set") + @Override + public void testResultSizeKeyRange() throws Exception { + } +} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java new file mode 100644 index 000000000..1c7765670 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java @@ -0,0 +1,96 @@ +/** + * 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 java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; + +import org.apache.gora.ignite.encoders.Encoder; +import org.apache.gora.ignite.encoders.SignedBinaryEncoder; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * + */ +public class PartitionTest { + // TODO test more types + + private static Encoder encoder = new SignedBinaryEncoder(); + + static long encl(long l) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + try { + dos.writeLong(l); + dos.flush(); + } catch (IOException e) { + throw new RuntimeException(e); + } + return encoder.decodeLong(baos.toByteArray()); + } + + @Test + public void test1() throws IOException { + assertEquals(encl(0x006f000000000000l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0x00, 0x6f})); + assertEquals(encl(1l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0, 0, 0, 0, 0, 0, 0, 0})); + assertEquals(encl(0x106f000000000001l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0x10, 0x6f, 0, 0, 0, 0, 0, 0})); + assertEquals( + encl(-1l), + (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + (byte) 0xff, + (byte) 0xfe})); + + assertEquals(encl(0x8000000000000001l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0x80, 0, 0, 0, 0, 0, 0, 0})); + assertEquals( + encl(0x8000000000000000l), + (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0x7f, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, + (byte) 0xff, + (byte) 0xff})); + + + try { + IgniteStore.followingKey(encoder, Long.class, + new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff}); + assertTrue(false); + } catch (IllegalArgumentException iea) { + + } + } + + @Test + public void test2() throws IOException { + assertEquals(encl(0x00ffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {0x01})); + assertEquals(encl(0x006effffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {0x00, 0x6f})); + assertEquals(encl(0xff6effffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0xff, 0x6f})); + assertEquals(encl(0xfffeffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0xff, (byte) 0xff})); + assertEquals(encl(0l), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0, 0, 0, 0, 0, 0, 0, 0})); + + assertEquals(encl(0x7effffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x7f})); + assertEquals(encl(0x7fffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x80})); + assertEquals(encl(0x80ffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x81})); + + try { + IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0, 0, 0, 0, 0, 0, 0}); + assertTrue(false); + } catch (IllegalArgumentException iea) { + + } + } +} 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 000000000..a41e0853f --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java @@ -0,0 +1,21 @@ +/* + * 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 unit tests for basic CRUD operations + * functionality of the Ignite dataStore. + */ +package org.apache.gora.ignite.store; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java new file mode 100644 index 000000000..0d4e5e662 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java @@ -0,0 +1,56 @@ +/** + * 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.util; + +import java.nio.charset.Charset; +import org.apache.gora.ignite.encoders.HexEncoder; +import static org.junit.Assert.assertEquals; +import org.junit.Test; + +/** + * + */ +public class HexEncoderTest { + + @Test + public void testByte() { + HexEncoder encoder = new HexEncoder(); + + assertEquals("12", new String(encoder.encodeByte((byte) 0x12), Charset.defaultCharset())); + assertEquals("f2", new String(encoder.encodeByte((byte) 0xf2), Charset.defaultCharset())); + + byte b = Byte.MIN_VALUE; + while (b != Byte.MAX_VALUE) { + assertEquals(b, encoder.decodeByte(encoder.encodeByte(b))); + b++; + } + } + + @Test + public void testShort() { + HexEncoder encoder = new HexEncoder(); + + assertEquals("1234", new String(encoder.encodeShort((short) 0x1234), Charset.defaultCharset())); + assertEquals("f234", new String(encoder.encodeShort((short) 0xf234), Charset.defaultCharset())); + + short s = Short.MIN_VALUE; + while (s != Short.MAX_VALUE) { + assertEquals(s, encoder.decodeShort(encoder.encodeShort(s))); + s++; + } + } +} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java new file mode 100644 index 000000000..c943e73b7 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java @@ -0,0 +1,167 @@ +/** + * 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.util; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; + +import org.apache.gora.ignite.encoders.SignedBinaryEncoder; +import org.apache.hadoop.io.Text; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** + * + */ +public class SignedBinaryEncoderTest { + @Test + public void testShort() throws IOException { + short s = Short.MIN_VALUE; + Text prev = null; + + SignedBinaryEncoder encoder = new SignedBinaryEncoder(); + + while (true) { + byte[] enc = encoder.encodeShort(s); + assertEquals(s, encoder.decodeShort(enc)); + Text current = new Text(enc); + if (prev != null) + assertTrue(prev.compareTo(current) < 0); + prev = current; + s++; + if (s == Short.MAX_VALUE) + break; + } + } + + private void testInt(int start, int finish) throws IOException { + int i = start; + Text prev = null; + + SignedBinaryEncoder encoder = new SignedBinaryEncoder(); + + while (true) { + byte[] enc = encoder.encodeInt(i); + assertEquals(i, encoder.decodeInt(enc)); + Text current = new Text(enc); + if (prev != null) + assertTrue(prev.compareTo(current) < 0); + prev = current; + i++; + if (i == finish) + break; + } + } + + @Test + public void testInt() throws IOException { + testInt(Integer.MIN_VALUE, Integer.MIN_VALUE + (1 << 16)); + testInt(-(1 << 15), (1 << 15)); + testInt(Integer.MAX_VALUE - (1 << 16), Integer.MAX_VALUE); + } + + private void testLong(long start, long finish) throws IOException { + long l = start; + Text prev = null; + + SignedBinaryEncoder encoder = new SignedBinaryEncoder(); + + while (true) { + byte[] enc = encoder.encodeLong(l); + assertEquals(l, encoder.decodeLong(enc)); + Text current = new Text(enc); + if (prev != null) + assertTrue(prev.compareTo(current) < 0); + prev = current; + l++; + if (l == finish) + break; + } + } + + @Test + public void testLong() throws IOException { + testLong(Long.MIN_VALUE, Long.MIN_VALUE + (1 << 16)); + testLong(-(1 << 15), (1 << 15)); + testLong(Long.MAX_VALUE - (1 << 16), Long.MAX_VALUE); + } + + @Test + public void testDouble() throws IOException { + + ArrayList testData = new ArrayList<>(); + testData.add(Double.NEGATIVE_INFINITY); + testData.add(Double.MIN_VALUE); + testData.add(Math.nextUp(Double.NEGATIVE_INFINITY)); + testData.add(Math.pow(10.0, 30.0) * -1.0); + testData.add(Math.pow(10.0, 30.0)); + testData.add(Math.pow(10.0, -30.0) * -1.0); + testData.add(Math.pow(10.0, -30.0)); + testData.add(Math.nextAfter(0.0, Double.NEGATIVE_INFINITY)); + testData.add(0.0); + testData.add(Math.nextAfter(Double.MAX_VALUE, Double.NEGATIVE_INFINITY)); + testData.add(Double.MAX_VALUE); + testData.add(Double.POSITIVE_INFINITY); + + Collections.sort(testData); + + SignedBinaryEncoder encoder = new SignedBinaryEncoder(); + + for (int i = 0; i < testData.size(); i++) { + byte[] enc = encoder.encodeDouble(testData.get(i)); + assertEquals(testData.get(i), (Double) encoder.decodeDouble(enc)); + if (i > 1) { + assertTrue("Checking " + testData.get(i) + " > " + testData.get(i - 1), + new Text(enc).compareTo(new Text(encoder.encodeDouble(testData.get(i - 1)))) > 0); + } + } + } + + @Test + public void testFloat() throws IOException { + + ArrayList testData = new ArrayList<>(); + testData.add(Float.NEGATIVE_INFINITY); + testData.add(Float.MIN_VALUE); + testData.add(Math.nextUp(Float.NEGATIVE_INFINITY)); + testData.add((float) Math.pow(10.0f, 30.0f) * -1.0f); + testData.add((float) Math.pow(10.0f, 30.0f)); + testData.add((float) Math.pow(10.0f, -30.0f) * -1.0f); + testData.add((float) Math.pow(10.0f, -30.0f)); + testData.add(Math.nextAfter(0.0f, Float.NEGATIVE_INFINITY)); + testData.add(0.0f); + testData.add(Math.nextAfter(Float.MAX_VALUE, Float.NEGATIVE_INFINITY)); + testData.add(Float.MAX_VALUE); + testData.add(Float.POSITIVE_INFINITY); + + Collections.sort(testData); + + SignedBinaryEncoder encoder = new SignedBinaryEncoder(); + + for (int i = 0; i < testData.size(); i++) { + byte[] enc = encoder.encodeFloat(testData.get(i)); + assertEquals(testData.get(i), (Float)encoder.decodeFloat(enc)); + if (i > 1) { + assertTrue("Checking " + testData.get(i) + " > " + testData.get(i - 1), + new Text(enc).compareTo(new Text(encoder.encodeFloat(testData.get(i - 1)))) > 0); + } + } + } + +} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java new file mode 100644 index 000000000..afcd61954 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/util/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 all the unit tests for utils of the Ignite dataStore. + */ +package org.apache.gora.ignite.util; \ No newline at end of file diff --git a/gora-ignite/src/test/resources/gora-accumulo-mapping.xml b/gora-ignite/src/test/resources/gora-accumulo-mapping.xml new file mode 100644 index 000000000..4d036c635 --- /dev/null +++ b/gora-ignite/src/test/resources/gora-accumulo-mapping.xml @@ -0,0 +1,59 @@ + + + + + + + +
+ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
diff --git a/gora-ignite/src/test/resources/gora.properties b/gora-ignite/src/test/resources/gora.properties new file mode 100644 index 000000000..5cf0e251c --- /dev/null +++ b/gora-ignite/src/test/resources/gora.properties @@ -0,0 +1,21 @@ +# 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. + +gora.datastore.default=org.apache.gora.ignite.store.IgniteStore +gora.datastore.ignite.mock=false +gora.datastore.ignite.instance=goraTest +gora.datastore.ignite.zookeepers=localhost:56321 +gora.datastore.ignite.user=root +gora.datastore.ignite.password=drowssap \ No newline at end of file diff --git a/pom.xml b/pom.xml index 48ff3b430..88b9b52da 100644 --- a/pom.xml +++ b/pom.xml @@ -760,6 +760,7 @@ gora-tutorial sources-dist gora-aerospike + gora-ignite From 7545170df8047ea6127660c1750e1d0b59044e49 Mon Sep 17 00:00:00 2001 From: Carlos M Date: Wed, 6 Jun 2018 00:03:10 -0500 Subject: [PATCH 2/8] 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. --- gora-ignite/pom.xml | 21 +- .../gora/ignite/encoders/BinaryEncoder.java | 191 ---- .../apache/gora/ignite/encoders/Encoder.java | 72 -- .../gora/ignite/encoders/HexEncoder.java | 204 ---- .../ignite/encoders/SignedBinaryEncoder.java | 110 -- .../apache/gora/ignite/encoders/Utils.java | 91 -- .../gora/ignite/encoders/package-info.java | 20 - .../apache/gora/ignite/query/IgniteQuery.java | 45 - .../gora/ignite/query/IgniteResult.java | 101 -- .../gora/ignite/query/package-info.java | 21 - .../org/apache/gora/ignite/store/Column.java | 70 ++ .../gora/ignite/store/IgniteMapping.java | 44 +- .../ignite/store/IgniteMappingBuilder.java | 107 ++ .../gora/ignite/store/IgniteParameters.java | 139 +++ .../apache/gora/ignite/store/IgniteStore.java | 1013 ++--------------- .../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 -- .../gora/ignite/store/PartitionTest.java | 96 -- .../gora/ignite/store/TestIgniteStore.java | 32 + .../gora/ignite/store/package-info.java | 21 - .../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 - .../test/resources/gora-ignite-mapping.xml | 42 + .../src/test/resources/gora.properties | 11 +- nbactions.xml | 13 + 31 files changed, 550 insertions(+), 2528 deletions(-) delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/BinaryEncoder.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Encoder.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/HexEncoder.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/SignedBinaryEncoder.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/Utils.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/encoders/package-info.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/Column.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteMappingBuilder.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/store/IgniteParameters.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java delete mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/TestIgniteStore.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java delete mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java delete mode 100644 gora-ignite/src/test/resources/gora-accumulo-mapping.xml create mode 100644 gora-ignite/src/test/resources/gora-ignite-mapping.xml create mode 100644 nbactions.xml diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml index ddd3a4a77..fa31195a8 100644 --- a/gora-ignite/pom.xml +++ b/gora-ignite/pom.xml @@ -50,7 +50,7 @@ - 1.7.1 + 2.4.0 * org.apache.gora.ignite*;version="${project.version}";-noimport:=true @@ -114,13 +114,6 @@ ${ignite.version} - - org.apache.ignite - ignite-minicluster - ${ignite.version} - test - - commons-io commons-io @@ -144,6 +137,12 @@ org.apache.avro avro + + + org.jdom + jdom + compile + @@ -171,6 +170,12 @@ org.apache.hadoop hadoop-minicluster + + + org.apache.ignite + ignite-indexing + 2.4.0 + 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 ce6eded29..000000000 --- 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); - } -} 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 deea4a756..000000000 --- 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); - -} 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 8568ba982..000000000 --- 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))); - } - -} 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 a8216f4bf..000000000 --- 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); - } - -} 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 8a5980c25..000000000 --- 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); - } - } -} 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 574aa24af..000000000 --- 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 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 85a59c9b7..000000000 --- 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 extends QueryBase { - - /** - * Constructor for the query - */ - public IgniteQuery() { - super(null); - } - - /** - * Constructor for the query - * - * @param dataStore Data store used - * - */ - public IgniteQuery(DataStore dataStore) { - super(dataStore); - } - -} 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 416e6503b..000000000 --- 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 extends ResultBase { - - private RowIterator iterator; - - /** - * Gets the data store used - */ - public IgniteStore getDataStore() { - return (IgniteStore) super.getDataStore(); - } - - /** - * @param dataStore - * @param query - * @param scanner - */ - public IgniteResult(DataStore dataStore, Query 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> nextRow = iterator.next(); - ByteSequence row = getDataStore().populate(nextRow, persistent); - key = ((IgniteStore) dataStore).fromBytes(getKeyClass(), row.toArray()); - - return true; - } - - @Override - public int size() { - return (int) this.limit; - } -} 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 b1a306c5e..000000000 --- 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 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 000000000..d06591e5e --- /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 + } +} 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 b46c06300..2e59da139 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> fieldMap = new HashMap<>(); + private String tableName; + private Map fields; + private List primaryKey; + + public IgniteMapping() { + fields = new HashMap<>(); + } + + public String getTableName() { + return tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public Map getFields() { + return fields; + } + + public void setFields(Map fields) { + this.fields = fields; + } - /** - * Look up the column associated to the Avro field. - */ - Map,String> columnMap = new HashMap<>(); + public List getPrimaryKey() { + return primaryKey; + } - Map tableConfig = new HashMap<>(); - String tableName; - String encoder; + public void setPrimaryKey(List primaryKey) { + this.primaryKey = primaryKey; + } } 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 000000000..81a3faedf --- /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 { + + private static final Logger LOG = LoggerFactory.getLogger(IgniteMappingBuilder.class); + /** + * Mapping instance being built + */ + private IgniteMapping igniteMapping; + + private final IgniteStore dataStore; + + public IgniteMappingBuilder(final IgniteStore 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 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 prColumns = classElement.getChildren("primarykey"); + List prFields = new ArrayList<>(); + for (Element aPrimaryKey : prColumns) { + prFields.add(aPrimaryKey.getAttributeValue("column")); + } + igniteMapping.setPrimaryKey(prFields); + List fields = classElement.getChildren("field"); + Map 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."); + + } +} 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 000000000..15b5baead --- /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)); + } + +} 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 2f5faf9e6..d52d10e20 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 @@ -16,1005 +16,165 @@ */ package org.apache.gora.ignite.store; -import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.net.InetAddress; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.text.MessageFormat; import java.util.List; -import java.util.Map; -import java.util.Map.Entry; +import java.util.Locale; import java.util.Properties; -import java.util.Set; -import java.util.concurrent.TimeUnit; - -import javax.xml.parsers.DocumentBuilder; -import javax.xml.parsers.DocumentBuilderFactory; - -import org.apache.ignite.core.client.IgniteException; -import org.apache.ignite.core.client.IgniteSecurityException; -import org.apache.ignite.core.client.BatchWriter; -import org.apache.ignite.core.client.BatchWriterConfig; -import org.apache.ignite.core.client.Connector; -import org.apache.ignite.core.client.IsolatedScanner; -import org.apache.ignite.core.client.IteratorSetting; -import org.apache.ignite.core.client.MutationsRejectedException; -import org.apache.ignite.core.client.RowIterator; -import org.apache.ignite.core.client.Scanner; -import org.apache.ignite.core.client.TableDeletedException; -import org.apache.ignite.core.client.TableExistsException; -import org.apache.ignite.core.client.TableNotFoundException; -import org.apache.ignite.core.client.TableOfflineException; -import org.apache.ignite.core.client.ZooKeeperInstance; -import org.apache.ignite.core.client.impl.ClientContext; -import org.apache.ignite.core.client.impl.Tables; -import org.apache.ignite.core.client.impl.TabletLocator; -import org.apache.ignite.core.client.mock.MockConnector; -import org.apache.ignite.core.client.mock.MockInstance; -import org.apache.ignite.core.client.mock.impl.MockTabletLocator; -import org.apache.ignite.core.client.security.tokens.AuthenticationToken; -import org.apache.ignite.core.client.security.tokens.PasswordToken; -import org.apache.ignite.core.conf.IgniteConfiguration; -import org.apache.ignite.core.data.ByteSequence; -import org.apache.ignite.core.data.Key; -import org.apache.ignite.core.data.impl.KeyExtent; -import org.apache.ignite.core.data.Mutation; -import org.apache.ignite.core.data.Range; -import org.apache.ignite.core.data.Value; -import org.apache.ignite.core.iterators.SortedKeyIterator; -import org.apache.ignite.core.iterators.user.TimestampFilter; -import org.apache.ignite.core.master.state.tables.TableState; -import org.apache.ignite.core.security.Authorizations; -import org.apache.ignite.core.security.ColumnVisibility; -import org.apache.ignite.core.client.impl.Credentials; -import org.apache.ignite.core.util.Pair; -import org.apache.ignite.core.util.UtilWaitThread; -import org.apache.avro.Schema; -import org.apache.avro.Schema.Field; -import org.apache.avro.Schema.Type; -import org.apache.avro.generic.GenericData; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.Decoder; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.io.EncoderFactory; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.avro.specific.SpecificDatumWriter; -import org.apache.avro.util.Utf8; -import org.apache.gora.ignite.encoders.BinaryEncoder; -import org.apache.gora.ignite.encoders.Encoder; -import org.apache.gora.ignite.query.IgniteQuery; -import org.apache.gora.ignite.query.IgniteResult; -import org.apache.gora.persistency.impl.DirtyListWrapper; -import org.apache.gora.persistency.impl.DirtyMapWrapper; +import java.util.logging.Level; import org.apache.gora.persistency.impl.PersistentBase; import org.apache.gora.query.PartitionQuery; import org.apache.gora.query.Query; import org.apache.gora.query.Result; -import org.apache.gora.query.impl.PartitionQueryImpl; -import org.apache.gora.store.DataStoreFactory; import org.apache.gora.store.impl.DataStoreBase; -import org.apache.gora.util.AvroUtils; import org.apache.gora.util.GoraException; -import org.apache.gora.util.IOUtils; -import org.apache.hadoop.io.Text; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.w3c.dom.Document; -import org.w3c.dom.Element; -import org.w3c.dom.NodeList; /** * Implementation of a Ignite data store to be used by gora. * - * @param - * class to be used for the key - * @param - * class to be persisted within the store + * @param class to be used for the key + * @param class to be persisted within the store */ -public class IgniteStore extends DataStoreBase { - - protected static final String MOCK_PROPERTY = "ignite.mock"; - protected static final String INSTANCE_NAME_PROPERTY = "ignite.instance"; - protected static final String ZOOKEEPERS_NAME_PROPERTY = "ignite.zookeepers"; - protected static final String USERNAME_PROPERTY = "ignite.user"; - protected static final String PASSWORD_PROPERTY = "ignite.password"; - protected static final String DEFAULT_MAPPING_FILE = "gora-ignite-mapping.xml"; - - private final static String UNKOWN = "Unknown type "; - - private Connector conn; - private BatchWriter batchWriter; - private IgniteMapping mapping; - private Credentials credentials; - private Encoder encoder; +public class IgniteStore extends DataStoreBase { public static final Logger LOG = LoggerFactory.getLogger(IgniteStore.class); + private static final String PARSE_MAPPING_FILE_KEY = "gora.ignite.mapping.file"; + private static final String DEFAULT_MAPPING_FILE = "gora-ignite-mapping.xml"; + private IgniteParameters igniteParameters; + private IgniteMapping igniteMapping; + private Connection connection; - public Object fromBytes(Schema schema, byte[] data) throws IOException { - Schema fromSchema = null; - if (schema.getType() == Type.UNION) { - try { - Decoder decoder = DecoderFactory.get().binaryDecoder(data, null); - int unionIndex = decoder.readIndex(); - List possibleTypes = schema.getTypes(); - fromSchema = possibleTypes.get(unionIndex); - Schema effectiveSchema = possibleTypes.get(unionIndex); - if (effectiveSchema.getType() == Type.NULL) { - decoder.readNull(); - return null; - } else { - data = decoder.readBytes(null).array(); - } - } catch (IOException e) { - LOG.error(e.getMessage()); - throw new GoraException("Error decoding union type: ", e); - } - } else { - fromSchema = schema; - } - return fromBytes(encoder, fromSchema, data); - } - - public static Object fromBytes(Encoder encoder, Schema schema, byte data[]) throws IOException { - switch (schema.getType()) { - case BOOLEAN: - return encoder.decodeBoolean(data); - case DOUBLE: - return encoder.decodeDouble(data); - case FLOAT: - return encoder.decodeFloat(data); - case INT: - return encoder.decodeInt(data); - case LONG: - return encoder.decodeLong(data); - case STRING: - return new Utf8(data); - case BYTES: - return ByteBuffer.wrap(data); - case ENUM: - return AvroUtils.getEnumValue(schema, encoder.decodeInt(data)); - case ARRAY: - break; - case FIXED: - break; - case MAP: - break; - case NULL: - break; - case RECORD: - break; - case UNION: - break; - default: - break; - } - throw new IllegalArgumentException(UNKOWN + schema.getType()); - - } - - private static byte[] getBytes(Text text) { - byte[] bytes = text.getBytes(); - if (bytes.length != text.getLength()) { - bytes = new byte[text.getLength()]; - System.arraycopy(text.getBytes(), 0, bytes, 0, bytes.length); - } - return bytes; - } - - public K fromBytes(Class clazz, byte[] val) { - return fromBytes(encoder, clazz, val); - } + @Override + public void initialize(Class keyClass, Class persistentClass, Properties properties) throws GoraException { - @SuppressWarnings("unchecked") - public static K fromBytes(Encoder encoder, Class clazz, byte[] val) { try { - if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { - return (K) Byte.valueOf(encoder.decodeByte(val)); - } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { - return (K) Boolean.valueOf(encoder.decodeBoolean(val)); - } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { - return (K) Short.valueOf(encoder.decodeShort(val)); - } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { - return (K) Integer.valueOf(encoder.decodeInt(val)); - } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { - return (K) Long.valueOf(encoder.decodeLong(val)); - } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { - return (K) Float.valueOf(encoder.decodeFloat(val)); - } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { - return (K) Double.valueOf(encoder.decodeDouble(val)); - } else if (clazz.equals(String.class)) { - return (K) new String(val, "UTF-8"); - } else if (clazz.equals(Utf8.class)) { - return (K) new Utf8(val); - } - - throw new IllegalArgumentException(UNKOWN + clazz.getName()); - } catch (IOException ioe) { - LOG.error(ioe.getMessage()); - throw new RuntimeException(ioe); + super.initialize(keyClass, persistentClass, properties); + IgniteMappingBuilder builder = new IgniteMappingBuilder(this); + builder.readMappingFile(getConf().get(PARSE_MAPPING_FILE_KEY, DEFAULT_MAPPING_FILE)); + igniteMapping = builder.getIgniteMapping(); + igniteParameters = IgniteParameters.load(properties, conf); + connection = acquiereConnection(); + LOG.info("Ignite store was successfully initialized"); + } catch (ClassNotFoundException | SQLException ex) { + LOG.error("Error while initializing Ignite store", ex); + throw new GoraException(ex); } } - private static byte[] copyIfNeeded(byte b[], int offset, int len) { - if (len != b.length || offset != 0) { - byte[] copy = new byte[len]; - System.arraycopy(b, offset, copy, 0, copy.length); - b = copy; + private Connection acquiereConnection() throws ClassNotFoundException, SQLException { + Class.forName("org.apache.ignite.IgniteJdbcThinDriver"); + StringBuilder urlBuilder = new StringBuilder(); + urlBuilder.append("jdbc:ignite:thin://"); + urlBuilder.append(igniteParameters.getHost()); + if (igniteParameters.getPort() != null) { + urlBuilder.append(":" + igniteParameters.getPort()); } - return b; - } - - public byte[] toBytes(Schema toSchema, Object o) { - if (toSchema != null && toSchema.getType() == Type.UNION) { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - org.apache.avro.io.BinaryEncoder avroEncoder = EncoderFactory.get().binaryEncoder(baos, null); - int unionIndex = 0; - try { - if (o == null) { - unionIndex = firstNullSchemaTypeIndex(toSchema); - avroEncoder.writeIndex(unionIndex); - avroEncoder.writeNull(); - } else { - unionIndex = firstNotNullSchemaTypeIndex(toSchema); - avroEncoder.writeIndex(unionIndex); - avroEncoder.writeBytes(toBytes(o)); - } - avroEncoder.flush(); - return baos.toByteArray(); - } catch (IOException e) { - LOG.error(e.getMessage()); - return toBytes(o); - } - } else { - return toBytes(o); + if (igniteParameters.getSchema() != null) { + urlBuilder.append("/" + igniteParameters.getSchema()); } - } - - private int firstNullSchemaTypeIndex(Schema toSchema) { - List possibleTypes = toSchema.getTypes(); - int unionIndex = 0; - for (int i = 0; i < possibleTypes.size(); i++ ) { - Type pType = possibleTypes.get(i).getType(); - if (pType == Type.NULL) { // FIXME HUGE kludge to pass tests - unionIndex = i; break; - } + if (igniteParameters.getUser() != null) { + urlBuilder.append(";" + igniteParameters.getUser()); } - return unionIndex; - } - - private int firstNotNullSchemaTypeIndex(Schema toSchema) { - List possibleTypes = toSchema.getTypes(); - int unionIndex = 0; - for (int i = 0; i < possibleTypes.size(); i++ ) { - Type pType = possibleTypes.get(i).getType(); - if (pType != Type.NULL) { // FIXME HUGE kludge to pass tests - unionIndex = i; break; - } + if (igniteParameters.getPassword() != null) { + urlBuilder.append(";" + igniteParameters.getPassword()); } - return unionIndex; - } - - public byte[] toBytes(Object o) { - return toBytes(encoder, o); - } - - public static byte[] toBytes(Encoder encoder, Object o) { - - try { - if (o instanceof String) { - return ((String) o).getBytes("UTF-8"); - } else if (o instanceof Utf8) { - return copyIfNeeded(((Utf8) o).getBytes(), 0, ((Utf8) o).getByteLength()); - } else if (o instanceof ByteBuffer) { - return copyIfNeeded(((ByteBuffer) o).array(), ((ByteBuffer) o).arrayOffset() + ((ByteBuffer) o).position(), ((ByteBuffer) o).remaining()); - } else if (o instanceof Long) { - return encoder.encodeLong((Long) o); - } else if (o instanceof Integer) { - return encoder.encodeInt((Integer) o); - } else if (o instanceof Short) { - return encoder.encodeShort((Short) o); - } else if (o instanceof Byte) { - return encoder.encodeByte((Byte) o); - } else if (o instanceof Boolean) { - return encoder.encodeBoolean((Boolean) o); - } else if (o instanceof Float) { - return encoder.encodeFloat((Float) o); - } else if (o instanceof Double) { - return encoder.encodeDouble((Double) o); - } else if (o instanceof Enum) { - return encoder.encodeInt(((Enum) o).ordinal()); - } - } catch (IOException ioe) { - throw new RuntimeException(ioe); + if (igniteParameters.getAdditionalConfigurations() != null) { + urlBuilder.append(igniteParameters.getAdditionalConfigurations()); } - - throw new IllegalArgumentException(UNKOWN + o.getClass().getName()); - } - - private BatchWriter getBatchWriter() throws IOException { - if (batchWriter == null) - try { - BatchWriterConfig batchWriterConfig = new BatchWriterConfig(); - batchWriterConfig.setMaxMemory(10000000); - batchWriterConfig.setMaxLatency(60000L, TimeUnit.MILLISECONDS); - batchWriterConfig.setMaxWriteThreads(4); - batchWriter = conn.createBatchWriter(mapping.tableName, batchWriterConfig); - } catch (TableNotFoundException e) { - throw new IOException(e); - } - return batchWriter; + Connection conn = DriverManager.getConnection(urlBuilder.toString()); + return conn; } - /** - * Initialize the data store by reading the credentials, setting the client's properties up and - * reading the mapping file. Initialize is called when then the call to - * {@link org.apache.gora.store.DataStoreFactory#createDataStore} is made. - * - * @param keyClass - * @param persistentClass - * @param properties - */ @Override - public void initialize(Class keyClass, Class persistentClass, Properties properties) throws GoraException { - super.initialize(keyClass, persistentClass, properties); - - try { - - String mock = DataStoreFactory.findProperty(properties, this, MOCK_PROPERTY, null); - String mappingFile = DataStoreFactory.getMappingFile(properties, this, DEFAULT_MAPPING_FILE); - String user = DataStoreFactory.findProperty(properties, this, USERNAME_PROPERTY, null); - String password = DataStoreFactory.findProperty(properties, this, PASSWORD_PROPERTY, null); - - mapping = readMapping(mappingFile); - - if (mapping.encoder == null || "".equals(mapping.encoder)) { - encoder = new BinaryEncoder(); - } else { - encoder = (Encoder) getClass().getClassLoader().loadClass(mapping.encoder).newInstance(); - } - - AuthenticationToken token = new PasswordToken(password); - if (mock == null || !mock.equals("true")) { - String instance = DataStoreFactory.findProperty(properties, this, INSTANCE_NAME_PROPERTY, null); - String zookeepers = DataStoreFactory.findProperty(properties, this, ZOOKEEPERS_NAME_PROPERTY, null); - conn = new ZooKeeperInstance(instance, zookeepers).getConnector(user, token); - } else { - conn = new MockInstance().getConnector(user, token); - } - credentials = new Credentials(user, token); - - if (autoCreateSchema && !schemaExists()) - createSchema(); - - } catch (IOException | InstantiationException | IllegalAccessException | - ClassNotFoundException | IgniteException | IgniteSecurityException e) { - throw new GoraException(e); - } - } - - protected IgniteMapping readMapping(String filename) throws IOException { - try { - - IgniteMapping mapping = new IgniteMapping(); - - DocumentBuilder db = DocumentBuilderFactory.newInstance().newDocumentBuilder(); - Document dom = db.parse(getClass().getClassLoader().getResourceAsStream(filename)); - - Element root = dom.getDocumentElement(); - - NodeList nl = root.getElementsByTagName("class"); - for (int i = 0; i < nl.getLength(); i++) { - - Element classElement = (Element) nl.item(i); - if (classElement.getAttribute("keyClass").equals(keyClass.getCanonicalName()) - && classElement.getAttribute("name").equals(persistentClass.getCanonicalName())) { - - mapping.tableName = getSchemaName(classElement.getAttribute("table"), persistentClass); - mapping.encoder = classElement.getAttribute("encoder"); - - NodeList fields = classElement.getElementsByTagName("field"); - for (int j = 0; j < fields.getLength(); j++) { - Element fieldElement = (Element) fields.item(j); - - String name = fieldElement.getAttribute("name"); - String family = fieldElement.getAttribute("family"); - String qualifier = fieldElement.getAttribute("qualifier"); - if ("".equals(qualifier)) - qualifier = null; - - Pair col = new Pair<>(new Text(family), qualifier == null ? null : new Text(qualifier)); - mapping.fieldMap.put(name, col); - mapping.columnMap.put(col, name); - } - } - - } - - if (mapping.tableName == null) { - throw new GoraException("Please define the ignite 'table' name mapping in " + filename + " for " + persistentClass.getCanonicalName()); - } - - nl = root.getElementsByTagName("table"); - for (int i = 0; i < nl.getLength(); i++) { - Element tableElement = (Element) nl.item(i); - if (tableElement.getAttribute("name").equals(mapping.tableName)) { - NodeList configs = tableElement.getElementsByTagName("config"); - for (int j = 0; j < configs.getLength(); j++) { - Element configElement = (Element) configs.item(j); - String key = configElement.getAttribute("key"); - String val = configElement.getAttribute("value"); - mapping.tableConfig.put(key, val); - } - } - } - - return mapping; - } catch (Exception ex) { - throw new IOException("Unable to read " + filename, ex); - } - + public String getSchemaName() { + return igniteMapping.getTableName(); } @Override - public String getSchemaName() { - return mapping.tableName; + public String getSchemaName(final String mappingSchemaName, + final Class persistentClass) { + return super.getSchemaName(mappingSchemaName, persistentClass); } @Override public void createSchema() throws GoraException { - try { - conn.tableOperations().create(mapping.tableName); - Set> es = mapping.tableConfig.entrySet(); - for (Entry entry : es) { - conn.tableOperations().setProperty(mapping.tableName, entry.getKey(), entry.getValue()); - } - - } catch (TableExistsException e) { - LOG.debug(e.getMessage(), e); - // Assume this is not an error - } catch (IgniteException | IgniteSecurityException e) { - throw new GoraException(e); - } + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override public void deleteSchema() throws GoraException { - try { - if (batchWriter != null) - batchWriter.close(); - batchWriter = null; - conn.tableOperations().delete(mapping.tableName); - } catch (TableNotFoundException e) { - // Ignore. Delete a non existant schema is a success - } catch (IgniteException | IgniteSecurityException e) { - throw new GoraException(e); - } + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override public boolean schemaExists() throws GoraException { - try { - return conn.tableOperations().exists(mapping.tableName); - } catch (Exception e) { - throw new GoraException(e); - } - } - - public ByteSequence populate(Iterator> iter, T persistent) throws IOException { - ByteSequence row = null; - - Map currentMap = null; - List currentArray = null; - Text currentFam = null; - int currentPos = 0; - Schema currentSchema = null; - Field currentField = null; - - BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(new byte[0], null); - - while (iter.hasNext()) { - Entry entry = iter.next(); - - if (row == null) { - row = entry.getKey().getRowData(); - } - byte[] val = entry.getValue().get(); - - Field field = fieldMap.get(getFieldName(entry)); - - if (currentMap != null) { - if (currentFam.equals(entry.getKey().getColumnFamily())) { - currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), - fromBytes(currentSchema, entry.getValue().get())); - continue; - } else { - persistent.put(currentPos, currentMap); - currentMap = null; - } - } else if (currentArray != null) { - if (currentFam.equals(entry.getKey().getColumnFamily())) { - currentArray.add(fromBytes(currentSchema, entry.getValue().get())); - continue; - } else { - persistent.put(currentPos, new GenericData.Array(currentField.schema(), currentArray)); - currentArray = null; - } - } - - switch (field.schema().getType()) { - case MAP: // first entry only. Next are handled above on the next loop - currentMap = new DirtyMapWrapper<>(new HashMap()); - currentPos = field.pos(); - currentFam = entry.getKey().getColumnFamily(); - currentSchema = field.schema().getValueType(); - - currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), - fromBytes(currentSchema, entry.getValue().get())); - break; - case ARRAY: - currentArray = new DirtyListWrapper<>(new ArrayList<>()); - currentPos = field.pos(); - currentFam = entry.getKey().getColumnFamily(); - currentSchema = field.schema().getElementType(); - currentField = field; - - currentArray.add(fromBytes(currentSchema, entry.getValue().get())); - - break; - case UNION:// default value of null acts like union with null - Schema effectiveSchema = field.schema().getTypes() - .get(firstNotNullSchemaTypeIndex(field.schema())); - // map and array were coded without union index so need to be read the same way - if (effectiveSchema.getType() == Type.ARRAY) { - currentArray = new DirtyListWrapper<>(new ArrayList<>()); - currentPos = field.pos(); - currentFam = entry.getKey().getColumnFamily(); - currentSchema = field.schema().getElementType(); - currentField = field; - - currentArray.add(fromBytes(currentSchema, entry.getValue().get())); - break; - } - else if (effectiveSchema.getType() == Type.MAP) { - currentMap = new DirtyMapWrapper<>(new HashMap()); - currentPos = field.pos(); - currentFam = entry.getKey().getColumnFamily(); - currentSchema = effectiveSchema.getValueType(); - - currentMap.put(new Utf8(entry.getKey().getColumnQualifierData().toArray()), - fromBytes(currentSchema, entry.getValue().get())); - break; - } - // continue like a regular top-level union - case RECORD: - SpecificDatumReader reader = new SpecificDatumReader(field.schema()); - persistent.put(field.pos(), reader.read(null, DecoderFactory.get().binaryDecoder(val, decoder))); - break; - default: - persistent.put(field.pos(), fromBytes(field.schema(), entry.getValue().get())); - } - } - - if (currentMap != null) { - persistent.put(currentPos, currentMap); - } else if (currentArray != null) { - persistent.put(currentPos, new GenericData.Array(currentField.schema(), currentArray)); - } - - persistent.clearDirty(); - - return row; - } - - /** - * Retrieve field name from entry. - * @param entry The Key-Value entry - * @return String The field name - */ - private String getFieldName(Entry entry) { - String fieldName = mapping.columnMap.get(new Pair<>(entry.getKey().getColumnFamily(), - entry.getKey().getColumnQualifier())); - if (fieldName == null) { - fieldName = mapping.columnMap.get(new Pair(entry.getKey().getColumnFamily(), null)); - } - return fieldName; - } - - private void setFetchColumns(Scanner scanner, String[] fields) { - fields = getFieldsToQuery(fields); - for (String field : fields) { - Pair col = mapping.fieldMap.get(field); - if (col != null) { - if (col.getSecond() == null) { - scanner.fetchColumnFamily(col.getFirst()); - } else { - scanner.fetchColumn(col.getFirst(), col.getSecond()); - } + boolean exists = false; + try (Statement stmt = connection.createStatement()) { + MessageFormat messageFormat = new MessageFormat("select * from {0} limit 0", Locale.getDefault()); + ResultSet executeQuery = stmt.executeQuery(messageFormat.format(igniteMapping.getTableName())); + executeQuery.close(); + exists = true; + } catch (SQLException ex) { + /** + * a 42000 error code is thrown by Ignite when a non-existent table + * queried. More details: + * https://apacheignite-sql.readme.io/docs/jdbc-error-codes + */ + if (ex.getSQLState() != null && ex.getSQLState().equals("42000")) { + exists = false; } else { - LOG.error("Mapping not found for field: {}", field); + throw new GoraException(ex); } } + return exists; } @Override public T get(K key, String[] fields) throws GoraException { - try { - // TODO make isolated scanner optional? - Scanner scanner = new IsolatedScanner(conn.createScanner(mapping.tableName, Authorizations.EMPTY)); - Range rowRange = new Range(new Text(toBytes(key))); - - scanner.setRange(rowRange); - setFetchColumns(scanner, fields); - - T persistent = newPersistent(); - ByteSequence row = populate(scanner.iterator(), persistent); - if (row == null) - return null; - return persistent; - } catch (Exception e) { - throw new GoraException(e); - } + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override - public void put(K key, T val) throws GoraException { - - try{ - Mutation m = new Mutation(new Text(toBytes(key))); - - Schema schema = val.getSchema(); - List fields = schema.getFields(); - int count = 0; - - for (int i = 0; i < fields.size(); i++) { - if (!val.isDirty(i)) { - continue; - } - Field field = fields.get(i); - - Object o = val.get(field.pos()); - - Pair col = mapping.fieldMap.get(field.name()); - - if (col == null) { - throw new GoraException("Please define the gora to ignite mapping for field " + field.name()); - } - - switch (field.schema().getType()) { - case MAP: - count = putMap(m, count, field.schema().getValueType(), o, col, field.name()); - break; - case ARRAY: - count = putArray(m, count, o, col, field.name()); - break; - case UNION: // default value of null acts like union with null - Schema effectiveSchema = field.schema().getTypes() - .get(firstNotNullSchemaTypeIndex(field.schema())); - // map and array need to compute qualifier - if (effectiveSchema.getType() == Type.ARRAY) { - count = putArray(m, count, o, col, field.name()); - break; - } - else if (effectiveSchema.getType() == Type.MAP) { - count = putMap(m, count, effectiveSchema.getValueType(), o, col, field.name()); - break; - } - // continue like a regular top-level union - case RECORD: - final SpecificDatumWriter writer = new SpecificDatumWriter<>(field.schema()); - final byte[] byteData = IOUtils.serialize(writer,o); - m.put(col.getFirst(), col.getSecond(), new Value(byteData)); - count++; - break; - default: - m.put(col.getFirst(), col.getSecond(), new Value(toBytes(o))); - count++; - } - - } - - if (count > 0) - try { - getBatchWriter().addMutation(m); - } catch (MutationsRejectedException e) { - LOG.error(e.getMessage(), e); - } - } catch (GoraException e) { - throw e; - } catch (Exception e) { - throw new GoraException(e); - } - } - - private int putMap(Mutation m, int count, Schema valueType, Object o, Pair col, String fieldName) throws GoraException { - - // First of all we delete map field on ignite store - Text rowKey = new Text(m.getRow()); - Query query = newQuery(); - query.setFields(fieldName); - query.setStartKey((K)rowKey.toString()); - query.setEndKey((K)rowKey.toString()); - deleteByQuery(query); - flush(); - if (o == null){ - return 0; - } - - Set es = ((Map)o).entrySet(); - for (Object entry : es) { - Object mapKey = ((Entry) entry).getKey(); - Object mapVal = ((Entry) entry).getValue(); - if ((o instanceof DirtyMapWrapper && ((DirtyMapWrapper)o).isDirty()) - || !(o instanceof DirtyMapWrapper)) { - m.put(col.getFirst(), new Text(toBytes(mapKey)), new Value(toBytes(valueType, mapVal))); - count++; - } - // TODO map value deletion - } - return count; - } - - private int putArray(Mutation m, int count, Object o, Pair col, String fieldName) throws GoraException { - - // First of all we delete array field on ignite store - Text rowKey = new Text(m.getRow()); - Query query = newQuery(); - query.setFields(fieldName); - query.setStartKey((K)rowKey.toString()); - query.setEndKey((K)rowKey.toString()); - deleteByQuery(query); - flush(); - if (o == null){ - return 0; - } - - List array = (List) o; // both GenericArray and DirtyListWrapper - int j = 0; - for (Object item : array) { - m.put(col.getFirst(), new Text(toBytes(j++)), new Value(toBytes(item))); - count++; - } - return count; + public void put(K key, T obj) throws GoraException { + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override public boolean delete(K key) throws GoraException { - Query q = newQuery(); - q.setKey(key); - return deleteByQuery(q) > 0; + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override - public long deleteByQuery(Query query) throws GoraException { - try { - Scanner scanner = createScanner(query); - // add iterator that drops values on the server side - scanner.addScanIterator(new IteratorSetting(Integer.MAX_VALUE, SortedKeyIterator.class)); - RowIterator iterator = new RowIterator(scanner.iterator()); - - long count = 0; - - while (iterator.hasNext()) { - Iterator> row = iterator.next(); - Mutation m = null; - while (row.hasNext()) { - Entry entry = row.next(); - Key key = entry.getKey(); - if (m == null) - m = new Mutation(key.getRow()); - // TODO optimize to avoid continually creating column vis? prob does not matter for empty - m.putDelete(key.getColumnFamily(), key.getColumnQualifier(), new ColumnVisibility(key.getColumnVisibility()), key.getTimestamp()); - } - getBatchWriter().addMutation(m); - count++; - } - - return count; - } catch (Exception e) { - throw new GoraException(e); - } + public long deleteByQuery(Query query) throws GoraException { + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } - private Range createRange(Query query) { - Text startRow = null; - Text endRow = null; - - if (query.getStartKey() != null) - startRow = new Text(toBytes(query.getStartKey())); - - if (query.getEndKey() != null) - endRow = new Text(toBytes(query.getEndKey())); - - return new Range(startRow, true, endRow, true); - - } - - private Scanner createScanner(Query query) throws TableNotFoundException { - // TODO make isolated scanner optional? - Scanner scanner = new IsolatedScanner(conn.createScanner(mapping.tableName, Authorizations.EMPTY)); - setFetchColumns(scanner, query.getFields()); - - scanner.setRange(createRange(query)); - - if (query.getStartTime() != -1 || query.getEndTime() != -1) { - IteratorSetting is = new IteratorSetting(30, TimestampFilter.class); - if (query.getStartTime() != -1) - TimestampFilter.setStart(is, query.getStartTime(), true); - if (query.getEndTime() != -1) - TimestampFilter.setEnd(is, query.getEndTime(), true); - - scanner.addScanIterator(is); - } - - return scanner; - } - - /** - * Execute the query and return the result. - */ @Override - public Result execute(Query query) throws GoraException { - try { - Scanner scanner = createScanner(query); - return new IgniteResult<>(this, query, scanner); - } catch (TableNotFoundException e) { - throw new GoraException(e) ; - } + public Result execute(Query query) throws GoraException { + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override - public Query newQuery() { - return new IgniteQuery<>(this); - } - - Text pad(Text key, int bytes) { - if (key.getLength() < bytes) - key = new Text(key); - - while (key.getLength() < bytes) { - key.append(new byte[] {0}, 0, 1); - } - - return key; + public Query newQuery() { + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override - public List> getPartitions(Query query) throws GoraException { - try { - TabletLocator tl; - if (conn instanceof MockConnector) - tl = new MockTabletLocator(); - else - tl = TabletLocator.getLocator(new ClientContext(conn.getInstance(), credentials, IgniteConfiguration.getTableConfiguration(conn, Tables.getTableId(conn.getInstance(), mapping.tableName))), new Text(Tables.getTableId(conn.getInstance(), mapping.tableName))); - - Map>> binnedRanges = new HashMap<>(); - - tl.invalidateCache(); - while (tl.binRanges(new ClientContext(conn.getInstance(), credentials, IgniteConfiguration.getTableConfiguration(conn, Tables.getTableId(conn.getInstance(), mapping.tableName))), Collections.singletonList(createRange(query)), binnedRanges).size() > 0) { - // TODO log? - if (!Tables.exists(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName))) - throw new TableDeletedException(Tables.getTableId(conn.getInstance(), mapping.tableName)); - else if (Tables.getTableState(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName)) == TableState.OFFLINE) - throw new TableOfflineException(conn.getInstance(), Tables.getTableId(conn.getInstance(), mapping.tableName)); - UtilWaitThread.sleep(100); - tl.invalidateCache(); - } - - List> ret = new ArrayList<>(); - - Text startRow = null; - Text endRow = null; - if (query.getStartKey() != null) - startRow = new Text(toBytes(query.getStartKey())); - if (query.getEndKey() != null) - endRow = new Text(toBytes(query.getEndKey())); - - //hadoop expects hostnames, ignite keeps track of IPs... so need to convert - HashMap hostNameCache = new HashMap<>(); - - for (Entry>> entry : binnedRanges.entrySet()) { - String ip = entry.getKey().split(":", 2)[0]; - String location = hostNameCache.get(ip); - if (location == null) { - InetAddress inetAddress = InetAddress.getByName(ip); - location = inetAddress.getHostName(); - hostNameCache.put(ip, location); - } - - Map> tablets = entry.getValue(); - for (KeyExtent ke : tablets.keySet()) { - - K startKey = null; - if (startRow == null || !ke.contains(startRow)) { - if (ke.getPrevEndRow() != null) { - startKey = followingKey(encoder, getKeyClass(), getBytes(ke.getPrevEndRow())); - } - } else { - startKey = fromBytes(getKeyClass(), getBytes(startRow)); - } - - K endKey = null; - if (endRow == null || !ke.contains(endRow)) { - if (ke.getEndRow() != null) - endKey = lastPossibleKey(encoder, getKeyClass(), getBytes(ke.getEndRow())); - } else { - endKey = fromBytes(getKeyClass(), getBytes(endRow)); - } - - PartitionQueryImpl pqi = new PartitionQueryImpl<>(query, startKey, endKey, location); - pqi.setConf(getConf()); - ret.add(pqi); - } - } - - return ret; - } catch (Exception e) { - throw new GoraException(e); - } - - } - - static K lastPossibleKey(Encoder encoder, Class clazz, byte[] er) { - - if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { - throw new UnsupportedOperationException(); - } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { - throw new UnsupportedOperationException(); - } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { - return fromBytes(encoder, clazz, encoder.lastPossibleKey(2, er)); - } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { - return fromBytes(encoder, clazz, encoder.lastPossibleKey(4, er)); - } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { - return fromBytes(encoder, clazz, encoder.lastPossibleKey(8, er)); - } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { - return fromBytes(encoder, clazz, encoder.lastPossibleKey(4, er)); - } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { - return fromBytes(encoder, clazz, encoder.lastPossibleKey(8, er)); - } else if (clazz.equals(String.class)) { - throw new UnsupportedOperationException(); - } else if (clazz.equals(Utf8.class)) { - return fromBytes(encoder, clazz, er); - } - - throw new IllegalArgumentException(UNKOWN + clazz.getName()); - } - - @SuppressWarnings("unchecked") - static K followingKey(Encoder encoder, Class clazz, byte[] per) { - - if (clazz.equals(Byte.TYPE) || clazz.equals(Byte.class)) { - return (K) Byte.valueOf(encoder.followingKey(1, per)[0]); - } else if (clazz.equals(Boolean.TYPE) || clazz.equals(Boolean.class)) { - throw new UnsupportedOperationException(); - } else if (clazz.equals(Short.TYPE) || clazz.equals(Short.class)) { - return fromBytes(encoder, clazz, encoder.followingKey(2, per)); - } else if (clazz.equals(Integer.TYPE) || clazz.equals(Integer.class)) { - return fromBytes(encoder, clazz, encoder.followingKey(4, per)); - } else if (clazz.equals(Long.TYPE) || clazz.equals(Long.class)) { - return fromBytes(encoder, clazz, encoder.followingKey(8, per)); - } else if (clazz.equals(Float.TYPE) || clazz.equals(Float.class)) { - return fromBytes(encoder, clazz, encoder.followingKey(4, per)); - } else if (clazz.equals(Double.TYPE) || clazz.equals(Double.class)) { - return fromBytes(encoder, clazz, encoder.followingKey(8, per)); - } else if (clazz.equals(String.class)) { - throw new UnsupportedOperationException(); - } else if (clazz.equals(Utf8.class)) { - return fromBytes(encoder, clazz, Arrays.copyOf(per, per.length + 1)); - } - - throw new IllegalArgumentException(UNKOWN + clazz.getName()); + public List> getPartitions(Query query) throws IOException { + throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. } @Override public void flush() throws GoraException { try { - if (batchWriter != null) { - batchWriter.flush(); - } + connection.commit(); } catch (Exception e) { throw new GoraException(e); } @@ -1023,12 +183,11 @@ public void flush() throws GoraException { @Override public void close() { try { - if (batchWriter != null) { - batchWriter.close(); - batchWriter = null; - } - } catch (MutationsRejectedException e) { - LOG.error(e.getMessage(), e); + connection.close(); + LOG.info("Ignite datastore destroyed successfully."); + } catch (Exception ex) { + LOG.error(ex.getMessage(), ex); } } + } diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java b/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.java deleted file mode 100644 index 97fb46ad1..000000000 --- a/gora-ignite/src/main/java/org/apache/gora/ignite/util/FixedByteArrayOutputStream.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.util; - -import java.io.IOException; -import java.io.OutputStream; - -/** - * It is a implementation of {@link java.io.OutputStream} must always provide at least a method that writes one byte of output. - */ -public class FixedByteArrayOutputStream extends OutputStream { - - private int i; - byte out[]; - - public FixedByteArrayOutputStream(byte out[]) { - this.out = out; - } - - @Override - public void write(int b) throws IOException { - out[i++] = (byte) b; - } - - @Override - public void write(byte b[], int off, int len) throws IOException { - System.arraycopy(b, off, out, i, len); - i += len; - } - -} \ No newline at end of file diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/util/package-info.java deleted file mode 100644 index eedb84b5c..000000000 --- a/gora-ignite/src/main/java/org/apache/gora/ignite/util/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. - */ -package org.apache.gora.ignite.util; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java b/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java index 1e0371731..336d81bb5 100644 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/GoraIgniteTestDriver.java @@ -1,4 +1,4 @@ -/** +/* * 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 @@ -17,57 +17,32 @@ */ package org.apache.gora.ignite; -import org.apache.ignite.minicluster.MiniIgniteCluster; -import org.apache.ignite.minicluster.MiniIgniteConfig; import org.apache.gora.GoraTestDriver; import org.apache.gora.ignite.store.IgniteStore; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; +import org.apache.ignite.Ignite; +import org.apache.ignite.Ignition; /** - * @author lmcgibbn + * Helper class for third part tests using gora-ignite backend. * + * @see GoraTestDriver */ public class GoraIgniteTestDriver extends GoraTestDriver { - private static final Logger LOG = LoggerFactory.getLogger(GoraIgniteTestDriver.class); - private static MiniIgniteCluster cluster = null; - private static final String PASSWORD = "drowssap"; + private Ignite igniteInstance; - @Rule - public TemporaryFolder tmpDir = new TemporaryFolder(); - - public GoraIgniteTestDriver() throws Exception { + public GoraIgniteTestDriver() { super(IgniteStore.class); } @Override - public void setUpClass() throws IOException, InterruptedException { - log.info("Starting Ignite MiniIgniteCluster..."); - try { - tmpDir.create(); - MiniIgniteConfig miniCfg = new MiniIgniteConfig(tmpDir.getRoot(), PASSWORD); - miniCfg.setInstanceName("goraTest"); - miniCfg.setZooKeeperPort(56321); - cluster = new MiniIgniteCluster(miniCfg); - cluster.start(); - } catch (Exception e) { - LOG.error("Error starting Ignite MiniIgniteCluster: {}", e.getMessage()); - // cleanup - tearDownClass(); - } + public void setUpClass() throws Exception { + igniteInstance = Ignition.start(); } @Override - public void tearDownClass() throws IOException, InterruptedException { - log.info("Shutting down Ignite MiniIgniteCluster..."); - if (cluster != null) { - cluster.stop(); - } - tmpDir.delete(); + public void tearDownClass() throws Exception { + igniteInstance.close(); } + } 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 deleted file mode 100644 index f536961c7..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/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. - */ -/** - * Tests for gora-ignite including - * the test driver for {@link org.apache.gora.ignite.store.IgniteStoreTest} - */ -package org.apache.gora.ignite; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java deleted file mode 100644 index bb77beea7..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/store/AuthenticationTokenTest.java +++ /dev/null @@ -1,90 +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.store; - -import java.util.List; -import java.util.Properties; -import org.apache.ignite.minicluster.MiniIgniteCluster; -import org.apache.gora.examples.generated.Employee; -import org.apache.gora.query.PartitionQuery; -import org.apache.gora.store.DataStore; -import org.apache.gora.store.DataStoreFactory; -import org.apache.hadoop.conf.Configuration; -import org.junit.After; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Tests authentication token is serialized correctly. - */ -public class AuthenticationTokenTest { - private static final Logger LOG = LoggerFactory.getLogger(AuthenticationTokenTest.class); - - private static final String GORA_DATASTORE = - DataStoreFactory.GORA + "." + DataStoreFactory.DATASTORE + "."; - private static final String PASSWORD = "password"; - - @Rule - public TemporaryFolder temporaryFolder = new TemporaryFolder(); - - private MiniIgniteCluster cluster; - private DataStore employeeStore; - - @Before - @SuppressWarnings("unchecked") - public void setUp() throws Exception { - cluster = new MiniIgniteCluster(temporaryFolder.getRoot(), PASSWORD); - cluster.start(); - - Properties properties = DataStoreFactory.createProps(); - properties.setProperty( - GORA_DATASTORE + IgniteStore.MOCK_PROPERTY, - "false"); - properties.setProperty( - GORA_DATASTORE + IgniteStore.INSTANCE_NAME_PROPERTY, - cluster.getInstanceName()); - properties.setProperty( - GORA_DATASTORE + IgniteStore.ZOOKEEPERS_NAME_PROPERTY, - cluster.getZooKeepers()); - properties.setProperty( - GORA_DATASTORE + IgniteStore.PASSWORD_PROPERTY, - PASSWORD); - - employeeStore = DataStoreFactory.createDataStore( - IgniteStore.class, - String.class, - Employee.class, - new Configuration(), - properties); - } - - @After - public void tearDown() throws Exception { - cluster.stop(); - } - - @Test - public void testAuthenticationTokenIsSerializedCorrectly() throws Exception { - List> partitions = - employeeStore.getPartitions(employeeStore.newQuery()); - LOG.debug("partitions {}", partitions); - } -} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java deleted file mode 100644 index b42d3442d..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/store/IgniteStoreTest.java +++ /dev/null @@ -1,87 +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.store; - -import java.io.IOException; - -import org.apache.gora.ignite.GoraIgniteTestDriver; -import org.apache.gora.examples.generated.Employee; -import org.apache.gora.examples.generated.WebPage; -import org.apache.gora.store.DataStore; -import org.apache.gora.store.DataStoreFactory; -import org.apache.gora.store.DataStoreTestBase; -import static org.apache.gora.store.DataStoreTestBase.log; -import org.apache.gora.store.DataStoreTestUtil; -import static org.apache.gora.store.DataStoreTestUtil.testResultSize; -import org.apache.hadoop.conf.Configuration; -import org.junit.Before; -import org.junit.Ignore; -import org.junit.Test; - -/** - * Tests extending {@link org.apache.gora.store.DataStoreTestBase} - * which run the base JUnit test suite for Gora. - */ -public class IgniteStoreTest extends DataStoreTestBase { - - static { - try { - setTestDriver(new GoraIgniteTestDriver()); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Before - public void setUp() throws Exception { - super.setUp(); - } - - public GoraIgniteTestDriver getTestDriver() { - return (GoraIgniteTestDriver) testDriver; - } - - //Until GORA-66 is resolved this test will always fail, so - //do not run it - @Ignore("skipped until GORA-66 is resolved") - @Override - public void testDeleteByQueryFields() throws IOException { - } - - @Test - @Ignore("Ignite does not support Result#size() without limit set") - @Override - public void testResultSize() throws Exception { - } - - @Test - @Ignore("Ignite does not support Result#size() without limit set") - @Override - public void testResultSizeStartKey() throws Exception { - } - - @Ignore("Ignite does not support Result#size() without limit set") - @Override - public void testResultSizeEndKey() throws Exception { - } - - @Test - @Ignore("Ignite does not support Result#size() without limit set") - @Override - public void testResultSizeKeyRange() throws Exception { - } -} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java deleted file mode 100644 index 1c7765670..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/store/PartitionTest.java +++ /dev/null @@ -1,96 +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.store; - -import java.io.ByteArrayOutputStream; -import java.io.DataOutputStream; -import java.io.IOException; - -import org.apache.gora.ignite.encoders.Encoder; -import org.apache.gora.ignite.encoders.SignedBinaryEncoder; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * - */ -public class PartitionTest { - // TODO test more types - - private static Encoder encoder = new SignedBinaryEncoder(); - - static long encl(long l) throws IOException { - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - DataOutputStream dos = new DataOutputStream(baos); - try { - dos.writeLong(l); - dos.flush(); - } catch (IOException e) { - throw new RuntimeException(e); - } - return encoder.decodeLong(baos.toByteArray()); - } - - @Test - public void test1() throws IOException { - assertEquals(encl(0x006f000000000000l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0x00, 0x6f})); - assertEquals(encl(1l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0, 0, 0, 0, 0, 0, 0, 0})); - assertEquals(encl(0x106f000000000001l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {0x10, 0x6f, 0, 0, 0, 0, 0, 0})); - assertEquals( - encl(-1l), - (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, - (byte) 0xff, - (byte) 0xfe})); - - assertEquals(encl(0x8000000000000001l), (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0x80, 0, 0, 0, 0, 0, 0, 0})); - assertEquals( - encl(0x8000000000000000l), - (long) IgniteStore.followingKey(encoder, Long.class, new byte[] {(byte) 0x7f, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, - (byte) 0xff, - (byte) 0xff})); - - - try { - IgniteStore.followingKey(encoder, Long.class, - new byte[] {(byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff, (byte) 0xff}); - assertTrue(false); - } catch (IllegalArgumentException iea) { - - } - } - - @Test - public void test2() throws IOException { - assertEquals(encl(0x00ffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {0x01})); - assertEquals(encl(0x006effffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {0x00, 0x6f})); - assertEquals(encl(0xff6effffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0xff, 0x6f})); - assertEquals(encl(0xfffeffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0xff, (byte) 0xff})); - assertEquals(encl(0l), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0, 0, 0, 0, 0, 0, 0, 0})); - - assertEquals(encl(0x7effffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x7f})); - assertEquals(encl(0x7fffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x80})); - assertEquals(encl(0x80ffffffffffffffl), (long) IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0x81})); - - try { - IgniteStore.lastPossibleKey(encoder, Long.class, new byte[] {(byte) 0, 0, 0, 0, 0, 0, 0}); - assertTrue(false); - } catch (IllegalArgumentException iea) { - - } - } -} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/store/TestIgniteStore.java b/gora-ignite/src/test/java/org/apache/gora/ignite/store/TestIgniteStore.java new file mode 100644 index 000000000..2244f0e21 --- /dev/null +++ b/gora-ignite/src/test/java/org/apache/gora/ignite/store/TestIgniteStore.java @@ -0,0 +1,32 @@ +/* + * 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 org.apache.gora.ignite.GoraIgniteTestDriver; +import org.apache.gora.store.DataStoreTestBase; + +/** + * Test case for IgniteStore. + */ +public class TestIgniteStore extends DataStoreTestBase { + + static { + setTestDriver(new GoraIgniteTestDriver()); + } + +} 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 deleted file mode 100644 index a41e0853f..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/store/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 unit tests for basic CRUD operations - * functionality of the Ignite dataStore. - */ -package org.apache.gora.ignite.store; \ No newline at end of file diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java deleted file mode 100644 index 0d4e5e662..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/util/HexEncoderTest.java +++ /dev/null @@ -1,56 +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.util; - -import java.nio.charset.Charset; -import org.apache.gora.ignite.encoders.HexEncoder; -import static org.junit.Assert.assertEquals; -import org.junit.Test; - -/** - * - */ -public class HexEncoderTest { - - @Test - public void testByte() { - HexEncoder encoder = new HexEncoder(); - - assertEquals("12", new String(encoder.encodeByte((byte) 0x12), Charset.defaultCharset())); - assertEquals("f2", new String(encoder.encodeByte((byte) 0xf2), Charset.defaultCharset())); - - byte b = Byte.MIN_VALUE; - while (b != Byte.MAX_VALUE) { - assertEquals(b, encoder.decodeByte(encoder.encodeByte(b))); - b++; - } - } - - @Test - public void testShort() { - HexEncoder encoder = new HexEncoder(); - - assertEquals("1234", new String(encoder.encodeShort((short) 0x1234), Charset.defaultCharset())); - assertEquals("f234", new String(encoder.encodeShort((short) 0xf234), Charset.defaultCharset())); - - short s = Short.MIN_VALUE; - while (s != Short.MAX_VALUE) { - assertEquals(s, encoder.decodeShort(encoder.encodeShort(s))); - s++; - } - } -} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java deleted file mode 100644 index c943e73b7..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/util/SignedBinaryEncoderTest.java +++ /dev/null @@ -1,167 +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.util; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; - -import org.apache.gora.ignite.encoders.SignedBinaryEncoder; -import org.apache.hadoop.io.Text; -import org.junit.Test; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -/** - * - */ -public class SignedBinaryEncoderTest { - @Test - public void testShort() throws IOException { - short s = Short.MIN_VALUE; - Text prev = null; - - SignedBinaryEncoder encoder = new SignedBinaryEncoder(); - - while (true) { - byte[] enc = encoder.encodeShort(s); - assertEquals(s, encoder.decodeShort(enc)); - Text current = new Text(enc); - if (prev != null) - assertTrue(prev.compareTo(current) < 0); - prev = current; - s++; - if (s == Short.MAX_VALUE) - break; - } - } - - private void testInt(int start, int finish) throws IOException { - int i = start; - Text prev = null; - - SignedBinaryEncoder encoder = new SignedBinaryEncoder(); - - while (true) { - byte[] enc = encoder.encodeInt(i); - assertEquals(i, encoder.decodeInt(enc)); - Text current = new Text(enc); - if (prev != null) - assertTrue(prev.compareTo(current) < 0); - prev = current; - i++; - if (i == finish) - break; - } - } - - @Test - public void testInt() throws IOException { - testInt(Integer.MIN_VALUE, Integer.MIN_VALUE + (1 << 16)); - testInt(-(1 << 15), (1 << 15)); - testInt(Integer.MAX_VALUE - (1 << 16), Integer.MAX_VALUE); - } - - private void testLong(long start, long finish) throws IOException { - long l = start; - Text prev = null; - - SignedBinaryEncoder encoder = new SignedBinaryEncoder(); - - while (true) { - byte[] enc = encoder.encodeLong(l); - assertEquals(l, encoder.decodeLong(enc)); - Text current = new Text(enc); - if (prev != null) - assertTrue(prev.compareTo(current) < 0); - prev = current; - l++; - if (l == finish) - break; - } - } - - @Test - public void testLong() throws IOException { - testLong(Long.MIN_VALUE, Long.MIN_VALUE + (1 << 16)); - testLong(-(1 << 15), (1 << 15)); - testLong(Long.MAX_VALUE - (1 << 16), Long.MAX_VALUE); - } - - @Test - public void testDouble() throws IOException { - - ArrayList testData = new ArrayList<>(); - testData.add(Double.NEGATIVE_INFINITY); - testData.add(Double.MIN_VALUE); - testData.add(Math.nextUp(Double.NEGATIVE_INFINITY)); - testData.add(Math.pow(10.0, 30.0) * -1.0); - testData.add(Math.pow(10.0, 30.0)); - testData.add(Math.pow(10.0, -30.0) * -1.0); - testData.add(Math.pow(10.0, -30.0)); - testData.add(Math.nextAfter(0.0, Double.NEGATIVE_INFINITY)); - testData.add(0.0); - testData.add(Math.nextAfter(Double.MAX_VALUE, Double.NEGATIVE_INFINITY)); - testData.add(Double.MAX_VALUE); - testData.add(Double.POSITIVE_INFINITY); - - Collections.sort(testData); - - SignedBinaryEncoder encoder = new SignedBinaryEncoder(); - - for (int i = 0; i < testData.size(); i++) { - byte[] enc = encoder.encodeDouble(testData.get(i)); - assertEquals(testData.get(i), (Double) encoder.decodeDouble(enc)); - if (i > 1) { - assertTrue("Checking " + testData.get(i) + " > " + testData.get(i - 1), - new Text(enc).compareTo(new Text(encoder.encodeDouble(testData.get(i - 1)))) > 0); - } - } - } - - @Test - public void testFloat() throws IOException { - - ArrayList testData = new ArrayList<>(); - testData.add(Float.NEGATIVE_INFINITY); - testData.add(Float.MIN_VALUE); - testData.add(Math.nextUp(Float.NEGATIVE_INFINITY)); - testData.add((float) Math.pow(10.0f, 30.0f) * -1.0f); - testData.add((float) Math.pow(10.0f, 30.0f)); - testData.add((float) Math.pow(10.0f, -30.0f) * -1.0f); - testData.add((float) Math.pow(10.0f, -30.0f)); - testData.add(Math.nextAfter(0.0f, Float.NEGATIVE_INFINITY)); - testData.add(0.0f); - testData.add(Math.nextAfter(Float.MAX_VALUE, Float.NEGATIVE_INFINITY)); - testData.add(Float.MAX_VALUE); - testData.add(Float.POSITIVE_INFINITY); - - Collections.sort(testData); - - SignedBinaryEncoder encoder = new SignedBinaryEncoder(); - - for (int i = 0; i < testData.size(); i++) { - byte[] enc = encoder.encodeFloat(testData.get(i)); - assertEquals(testData.get(i), (Float)encoder.decodeFloat(enc)); - if (i > 1) { - assertTrue("Checking " + testData.get(i) + " > " + testData.get(i - 1), - new Text(enc).compareTo(new Text(encoder.encodeFloat(testData.get(i - 1)))) > 0); - } - } - } - -} diff --git a/gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java b/gora-ignite/src/test/java/org/apache/gora/ignite/util/package-info.java deleted file mode 100644 index afcd61954..000000000 --- a/gora-ignite/src/test/java/org/apache/gora/ignite/util/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 all the unit tests for utils of the Ignite dataStore. - */ -package org.apache.gora.ignite.util; \ No newline at end of file diff --git a/gora-ignite/src/test/resources/gora-accumulo-mapping.xml b/gora-ignite/src/test/resources/gora-accumulo-mapping.xml deleted file mode 100644 index 4d036c635..000000000 --- a/gora-ignite/src/test/resources/gora-accumulo-mapping.xml +++ /dev/null @@ -1,59 +0,0 @@ - - - - - - - -
- - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
diff --git a/gora-ignite/src/test/resources/gora-ignite-mapping.xml b/gora-ignite/src/test/resources/gora-ignite-mapping.xml new file mode 100644 index 000000000..3c1377463 --- /dev/null +++ b/gora-ignite/src/test/resources/gora-ignite-mapping.xml @@ -0,0 +1,42 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/gora-ignite/src/test/resources/gora.properties b/gora-ignite/src/test/resources/gora.properties index 5cf0e251c..e45aab721 100644 --- a/gora-ignite/src/test/resources/gora.properties +++ b/gora-ignite/src/test/resources/gora.properties @@ -14,8 +14,9 @@ # limitations under the License. gora.datastore.default=org.apache.gora.ignite.store.IgniteStore -gora.datastore.ignite.mock=false -gora.datastore.ignite.instance=goraTest -gora.datastore.ignite.zookeepers=localhost:56321 -gora.datastore.ignite.user=root -gora.datastore.ignite.password=drowssap \ No newline at end of file +gora.datastore.ignite.schema=PUBLIC +gora.datastore.ignite.host=localhost +gora.datastore.ignite.port=10800 +gora.datastore.ignite.user= +gora.datastore.ignite.password= +gora.datastore.ignite.additionalConfigurations= \ No newline at end of file diff --git a/nbactions.xml b/nbactions.xml new file mode 100644 index 000000000..52ecddadc --- /dev/null +++ b/nbactions.xml @@ -0,0 +1,13 @@ + + + + rebuild + + * + + + clean + install + + + From b1e2ae4c902691bbce02a90b76c4bee1c6e58455 Mon Sep 17 00:00:00 2001 From: Carlos M Date: Tue, 26 Jun 2018 23:18:19 -0500 Subject: [PATCH 3/8] Implement schema handling methods The basic schema creation/deletion methods are implemented for the Ignite backend. In addition, all the SQL generation code was moved to a separate class. --- .../apache/gora/ignite/store/IgniteStore.java | 37 ++++++++-- .../gora/ignite/utils/IgniteSQLBuilder.java | 69 +++++++++++++++++++ .../src/test/resources/gora.properties | 4 +- 3 files changed, 101 insertions(+), 9 deletions(-) create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.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 d52d10e20..ebc2943f1 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 @@ -22,11 +22,9 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; -import java.text.MessageFormat; import java.util.List; -import java.util.Locale; import java.util.Properties; -import java.util.logging.Level; +import org.apache.gora.ignite.utils.IgniteSQLBuilder; import org.apache.gora.persistency.impl.PersistentBase; import org.apache.gora.query.PartitionQuery; import org.apache.gora.query.Query; @@ -105,20 +103,45 @@ public String getSchemaName(final String mappingSchemaName, @Override public void createSchema() throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + if (connection == null) { + throw new GoraException( + "Impossible to create the schema as no connection has been initiated."); + } + if (schemaExists()) { + return; + } + try (Statement stmt = connection.createStatement()) { + String createTableSQL = IgniteSQLBuilder.createTable(igniteMapping); + stmt.executeUpdate(createTableSQL); + LOG.info("Table {} has been created for Ignite instance.", + igniteMapping.getTableName()); + } catch (SQLException ex) { + throw new GoraException(ex); + } } @Override public void deleteSchema() throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + if (connection == null) { + throw new GoraException( + "Impossible to delete the schema as no connection has been initiated."); + } + try (Statement stmt = connection.createStatement()) { + String dropTableSQL = IgniteSQLBuilder.dropTable(igniteMapping.getTableName()); + stmt.executeUpdate(dropTableSQL); + LOG.info("Table {} has been dropped from Ignite instance.", + igniteMapping.getTableName()); + } catch (SQLException ex) { + throw new GoraException(ex); + } } @Override public boolean schemaExists() throws GoraException { boolean exists = false; try (Statement stmt = connection.createStatement()) { - MessageFormat messageFormat = new MessageFormat("select * from {0} limit 0", Locale.getDefault()); - ResultSet executeQuery = stmt.executeQuery(messageFormat.format(igniteMapping.getTableName())); + String tableExistsSQL = IgniteSQLBuilder.tableExists(igniteMapping.getTableName()); + ResultSet executeQuery = stmt.executeQuery(tableExistsSQL); executeQuery.close(); exists = true; } catch (SQLException ex) { 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 new file mode 100644 index 000000000..0480fd4e7 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteSQLBuilder.java @@ -0,0 +1,69 @@ +/** + * 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.utils; + +import avro.shaded.com.google.common.collect.Lists; +import java.text.MessageFormat; +import java.util.ArrayList; +import java.util.Locale; +import java.util.Map; +import org.apache.gora.ignite.store.Column; +import org.apache.gora.ignite.store.IgniteMapping; + +/** + * + * SQL Builder utility for Ignite. + */ +public class IgniteSQLBuilder { + + private static String format(String pattern, Object... args) { + MessageFormat messageFormat = new MessageFormat(pattern, Locale.getDefault()); + return messageFormat.format(args); + } + + public static String tableExists(String tableName) { + return format("SELECT * FROM {0} LIMIT 0", tableName); + } + + public static String createTable(IgniteMapping mapping) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("CREATE TABLE "); + sqlBuilder.append(mapping.getTableName()); + sqlBuilder.append("("); + ArrayList> fieldsList = Lists.newArrayList(mapping.getFields().entrySet()); + for (Map.Entry aField : fieldsList) { + Column aColumn = aField.getValue(); + String name = aColumn.getName(); + Column.FieldType dataType = aColumn.getDataType(); + sqlBuilder.append(name).append(" ").append(dataType.toString()).append(","); + } + sqlBuilder.append("PRIMARY KEY "); + sqlBuilder.append("("); + for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { + sqlBuilder.append(mapping.getPrimaryKey().get(i)); + sqlBuilder.append(i == mapping.getPrimaryKey().size() - 1 ? "" : ","); + } + sqlBuilder.append(")"); + sqlBuilder.append(");"); + return sqlBuilder.toString(); + } + + public static String dropTable(String tableName) { + return format("DROP TABLE IF EXISTS {0} ;", tableName); + } + +} diff --git a/gora-ignite/src/test/resources/gora.properties b/gora-ignite/src/test/resources/gora.properties index e45aab721..e0d190109 100644 --- a/gora-ignite/src/test/resources/gora.properties +++ b/gora-ignite/src/test/resources/gora.properties @@ -17,6 +17,6 @@ gora.datastore.default=org.apache.gora.ignite.store.IgniteStore gora.datastore.ignite.schema=PUBLIC gora.datastore.ignite.host=localhost gora.datastore.ignite.port=10800 -gora.datastore.ignite.user= -gora.datastore.ignite.password= +#gora.datastore.ignite.user= +#gora.datastore.ignite.password= gora.datastore.ignite.additionalConfigurations= \ No newline at end of file From db7540e045fc07ef39d37163a263615c6e580efe Mon Sep 17 00:00:00 2001 From: Carlos M Date: Mon, 2 Jul 2018 00:50:43 -0500 Subject: [PATCH 4/8] Implement put, get, delete The operations put, get and delete are implemented for the Ignite backend. In addition, primary key mapping is improved. --- .../gora/ignite/store/IgniteMapping.java | 6 +- .../ignite/store/IgniteMappingBuilder.java | 6 +- .../apache/gora/ignite/store/IgniteStore.java | 323 +++++++++++++++++- .../gora/ignite/utils/IgniteSQLBuilder.java | 90 ++++- .../test/resources/gora-ignite-mapping.xml | 10 +- 5 files changed, 412 insertions(+), 23 deletions(-) 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 2e59da139..c0e7a98d5 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 @@ -27,7 +27,7 @@ public class IgniteMapping { private String tableName; private Map fields; - private List primaryKey; + private List primaryKey; public IgniteMapping() { fields = new HashMap<>(); @@ -49,11 +49,11 @@ public void setFields(Map fields) { this.fields = fields; } - public List getPrimaryKey() { + public List getPrimaryKey() { return primaryKey; } - public void setPrimaryKey(List primaryKey) { + public void setPrimaryKey(List primaryKey) { this.primaryKey = primaryKey; } 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 81a3faedf..3d245ea5a 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 @@ -80,9 +80,11 @@ public void readMappingFile(String mappingFile) { String tableName = dataStore.getSchemaName(tableNameFromMapping, dataStore.getPersistentClass()); igniteMapping.setTableName(tableName); List prColumns = classElement.getChildren("primarykey"); - List prFields = new ArrayList<>(); + List prFields = new ArrayList<>(); for (Element aPrimaryKey : prColumns) { - prFields.add(aPrimaryKey.getAttributeValue("column")); + String name = aPrimaryKey.getAttributeValue("column"); + String type = aPrimaryKey.getAttributeValue("type"); + prFields.add(new Column(name, Column.FieldType.valueOf(type))); } igniteMapping.setPrimaryKey(prFields); List fields = classElement.getChildren("field"); 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 ebc2943f1..497e1612d 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 @@ -17,20 +17,34 @@ package org.apache.gora.ignite.store; import java.io.IOException; +import java.nio.ByteBuffer; import java.sql.Connection; import java.sql.DriverManager; +import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import java.util.Properties; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.avro.Schema; +import org.apache.avro.specific.SpecificDatumReader; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.util.Utf8; import org.apache.gora.ignite.utils.IgniteSQLBuilder; +import org.apache.gora.persistency.Persistent; import org.apache.gora.persistency.impl.PersistentBase; import org.apache.gora.query.PartitionQuery; import org.apache.gora.query.Query; import org.apache.gora.query.Result; import org.apache.gora.store.impl.DataStoreBase; +import org.apache.gora.util.AvroUtils; import org.apache.gora.util.GoraException; +import org.apache.gora.util.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,6 +63,16 @@ public class IgniteStore extends DataStoreBase> readerMap = new ConcurrentHashMap<>(); + + public static final ConcurrentHashMap> writerMap = new ConcurrentHashMap<>(); + @Override public void initialize(Class keyClass, Class persistentClass, Properties properties) throws GoraException { @@ -161,17 +185,151 @@ public boolean schemaExists() throws GoraException { @Override public T get(K key, String[] fields) throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + String[] avFields = getFieldsToQuery(fields); + Object[] keyl = null; + if (igniteMapping.getPrimaryKey().size() == 1) { + keyl = new Object[]{key}; + } else { + //Composite key pending + } + //Avro fields to Ignite fields + List dbFields = new ArrayList<>(); + for (String af : avFields) { + dbFields.add(igniteMapping.getFields().get(af).getName()); + } + String selectQuery = IgniteSQLBuilder.selectGet(igniteMapping, dbFields); + try (PreparedStatement stmt = connection.prepareStatement(selectQuery)) { + IgniteSQLBuilder.fillSelectStatement(stmt, igniteMapping, keyl); + ResultSet rs = stmt.executeQuery(); + boolean data = rs.next(); + T resp = null; + if (data) { + resp = newInstance(rs, fields); + if (rs.next()) { + LOG.warn("Multiple results for primary key {} in the schema {}, ignoring additional rows.", keyl, igniteMapping.getTableName()); + } + } + rs.close(); + return resp; + } catch (SQLException | IOException ex) { + throw new GoraException(ex); + } + + } + + public T newInstance(ResultSet rs, 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) { + continue; + } + Object v = deserializeFieldValue(field, fieldSchema, sv, persistent); + persistent.put(field.pos(), v); + persistent.setDirty(field.pos()); + } + return persistent; + } + + private Object deserializeFieldValue(Schema.Field field, Schema fieldSchema, + Object igniteValue, T persistent) throws IOException { + Object fieldValue = null; + switch (fieldSchema.getType()) { + case MAP: + case ARRAY: + case RECORD: + @SuppressWarnings("rawtypes") SpecificDatumReader reader = getDatumReader(fieldSchema); + fieldValue = IOUtils.deserialize((byte[]) igniteValue, reader, + persistent.get(field.pos())); + break; + case ENUM: + fieldValue = AvroUtils.getEnumValue(fieldSchema, igniteValue.toString()); + break; + case FIXED: + break; + case BYTES: + fieldValue = ByteBuffer.wrap((byte[]) igniteValue); + break; + case STRING: + fieldValue = new Utf8(igniteValue.toString()); + break; + case UNION: + if (fieldSchema.getTypes().size() == 2 && isNullable(fieldSchema)) { + int schemaPos = getUnionSchema(igniteValue, fieldSchema); + Schema unionSchema = fieldSchema.getTypes().get(schemaPos); + fieldValue = deserializeFieldValue(field, unionSchema, igniteValue, persistent); + } else { + reader = getDatumReader(fieldSchema); + fieldValue = IOUtils.deserialize((byte[]) igniteValue, reader, + persistent.get(field.pos())); + } + break; + default: + fieldValue = igniteValue; + } + return fieldValue; + } @Override public void put(K key, T obj) throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + try { + if (obj.isDirty()) { + Schema schema = obj.getSchema(); + List fields = schema.getFields(); + Map data = new HashMap<>(); + if (igniteMapping.getPrimaryKey().size() == 1) { + Column getKey = igniteMapping.getPrimaryKey().get(0); + data.put(getKey, key); + } else { + //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) { + Schema fieldSchema = field.schema(); + Object serializedObj = serializeFieldValue(get, fieldSchema, v); + data.put(get, serializedObj); + } + } + String baseInsertStatement = IgniteSQLBuilder.baseInsertStatement(igniteMapping, data); + try (PreparedStatement stmt = connection.prepareStatement(baseInsertStatement)) { + IgniteSQLBuilder.fillInsertStatement(stmt, data); + stmt.executeUpdate(); + } catch (SQLException ex) { + throw new GoraException(ex); + } + } else { + LOG.info("Ignored putting object {} in the store as it is neither " + + "new, neither dirty.", new Object[]{obj}); + } + } catch (Exception e) { + throw new GoraException(e); + } } @Override public boolean delete(K key) throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + String deleteQuery = null; + Object[] keyl = null; + if (igniteMapping.getPrimaryKey().size() == 1) { + deleteQuery = IgniteSQLBuilder.delete(igniteMapping); + keyl = new Object[]{key}; + } else { + //Composite key pending + } + try (PreparedStatement stmt = connection.prepareStatement(deleteQuery)) { + IgniteSQLBuilder.fillDeleteStatement(stmt, igniteMapping, keyl); + stmt.executeUpdate(); + return true; + } catch (SQLException ex) { + throw new GoraException(ex); + } } @Override @@ -196,11 +354,7 @@ public List> getPartitions(Query query) throws IOExce @Override public void flush() throws GoraException { - try { - connection.commit(); - } catch (Exception e) { - throw new GoraException(e); - } + //Auto-commit mode by default } @Override @@ -213,4 +367,157 @@ public void close() { } } + private Object serializeFieldValue(Column get, Schema fieldSchema, Object fieldValue) { + Object output = fieldValue; + switch (fieldSchema.getType()) { + case ARRAY: + case MAP: + case RECORD: + byte[] data = null; + try { + @SuppressWarnings("rawtypes") + SpecificDatumWriter writer = getDatumWriter(fieldSchema); + data = IOUtils.serialize(writer, fieldValue); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + output = data; + break; + case UNION: + if (fieldSchema.getTypes().size() == 2 && isNullable(fieldSchema)) { + int schemaPos = getUnionSchema(fieldValue, fieldSchema); + Schema unionSchema = fieldSchema.getTypes().get(schemaPos); + output = serializeFieldValue(get, unionSchema, fieldValue); + } else { + data = null; + try { + @SuppressWarnings("rawtypes") + SpecificDatumWriter writer = getDatumWriter(fieldSchema); + data = IOUtils.serialize(writer, fieldValue); + } catch (IOException e) { + LOG.error(e.getMessage(), e); + } + output = data; + } + break; + case FIXED: + break; + case ENUM: + case STRING: + output = fieldValue.toString(); + break; + case BYTES: + output = ((ByteBuffer) fieldValue).array(); + break; + case INT: + case LONG: + case FLOAT: + case DOUBLE: + case BOOLEAN: + output = fieldValue; + break; + case NULL: + break; + default: + throw new AssertionError(fieldSchema.getType().name()); + + } + return output; + } + + private boolean isNullable(Schema unionSchema) { + for (Schema innerSchema : unionSchema.getTypes()) { + if (innerSchema.getType().equals(Schema.Type.NULL)) { + return true; + } + } + return false; + } + + /** + * Method to retrieve the corresponding schema type index of a particular + * object having UNION schema. As UNION type can have one or more types and at + * a given instance, it holds an object of only one type of the defined types, + * this method is used to figure out the corresponding instance's schema type + * index. + * + * @param instanceValue value that the object holds + * @param unionSchema union schema containing all of the data types + * @return the unionSchemaPosition corresponding schema position + */ + private int getUnionSchema(Object instanceValue, Schema unionSchema) { + int unionSchemaPos = 0; + for (Schema currentSchema : unionSchema.getTypes()) { + Schema.Type schemaType = currentSchema.getType(); + if (instanceValue instanceof CharSequence && schemaType.equals(Schema.Type.STRING)) { + return unionSchemaPos; + } + if (instanceValue instanceof ByteBuffer && schemaType.equals(Schema.Type.BYTES)) { + return unionSchemaPos; + } + if (instanceValue instanceof byte[] && schemaType.equals(Schema.Type.BYTES)) { + return unionSchemaPos; + } + if (instanceValue instanceof Integer && schemaType.equals(Schema.Type.INT)) { + return unionSchemaPos; + } + if (instanceValue instanceof Long && schemaType.equals(Schema.Type.LONG)) { + return unionSchemaPos; + } + if (instanceValue instanceof Double && schemaType.equals(Schema.Type.DOUBLE)) { + return unionSchemaPos; + } + if (instanceValue instanceof Float && schemaType.equals(Schema.Type.FLOAT)) { + return unionSchemaPos; + } + if (instanceValue instanceof Boolean && schemaType.equals(Schema.Type.BOOLEAN)) { + return unionSchemaPos; + } + if (instanceValue instanceof Map && schemaType.equals(Schema.Type.MAP)) { + return unionSchemaPos; + } + if (instanceValue instanceof List && schemaType.equals(Schema.Type.ARRAY)) { + return unionSchemaPos; + } + if (instanceValue instanceof Persistent && schemaType.equals(Schema.Type.RECORD)) { + return unionSchemaPos; + } + if (instanceValue instanceof byte[] && schemaType.equals(Schema.Type.MAP)) { + return unionSchemaPos; + } + if (instanceValue instanceof byte[] && schemaType.equals(Schema.Type.RECORD)) { + return unionSchemaPos; + } + if (instanceValue instanceof byte[] && schemaType.equals(Schema.Type.ARRAY)) { + return unionSchemaPos; + } + unionSchemaPos++; + } + return 0; + } + + @SuppressWarnings("rawtypes") + private SpecificDatumReader getDatumReader(Schema fieldSchema) { + SpecificDatumReader reader = readerMap.get(fieldSchema); + if (reader == null) { + reader = new SpecificDatumReader(fieldSchema);// ignore dirty bits + SpecificDatumReader localReader = null; + if ((localReader = readerMap.putIfAbsent(fieldSchema, reader)) != null) { + reader = localReader; + } + } + return reader; + } + + @SuppressWarnings("rawtypes") + private SpecificDatumWriter getDatumWriter(Schema fieldSchema) { + SpecificDatumWriter writer = writerMap.get(fieldSchema); + if (writer == null) { + writer = new SpecificDatumWriter(fieldSchema);// ignore dirty bits + writerMap.put(fieldSchema, writer); + } + + return writer; + } + } 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 0480fd4e7..92014c190 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,10 +17,15 @@ package org.apache.gora.ignite.utils; import avro.shaded.com.google.common.collect.Lists; +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 java.util.Set; import org.apache.gora.ignite.store.Column; import org.apache.gora.ignite.store.IgniteMapping; @@ -44,9 +49,9 @@ public static String createTable(IgniteMapping mapping) { sqlBuilder.append("CREATE TABLE "); sqlBuilder.append(mapping.getTableName()); sqlBuilder.append("("); - ArrayList> fieldsList = Lists.newArrayList(mapping.getFields().entrySet()); - for (Map.Entry aField : fieldsList) { - Column aColumn = aField.getValue(); + ArrayList 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(","); @@ -54,16 +59,91 @@ public static String createTable(IgniteMapping mapping) { sqlBuilder.append("PRIMARY KEY "); sqlBuilder.append("("); for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { - sqlBuilder.append(mapping.getPrimaryKey().get(i)); + sqlBuilder.append(mapping.getPrimaryKey().get(i).getName()); sqlBuilder.append(i == mapping.getPrimaryKey().size() - 1 ? "" : ","); } sqlBuilder.append(")"); sqlBuilder.append(");"); return sqlBuilder.toString(); } - + public static String dropTable(String tableName) { return format("DROP TABLE IF EXISTS {0} ;", tableName); } + public static String baseInsertStatement(IgniteMapping mapping, Map data) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("MERGE INTO "); + sqlBuilder.append(mapping.getTableName()); + sqlBuilder.append(" ("); + List> list = new ArrayList<>(data.entrySet()); + 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 ? "" : ","); + } + sqlBuilder.append(" )"); + return sqlBuilder.toString(); + } + + public static void fillInsertStatement(PreparedStatement st, Map data) throws SQLException { + List> list = new ArrayList<>(data.entrySet()); + for (int i = 0; i < list.size(); i++) { + int j = i + 1; + st.setObject(j, list.get(i).getValue()); + } + } + + public static String delete(IgniteMapping mapping) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("DELETE FROM "); + sqlBuilder.append(mapping.getTableName()); + sqlBuilder.append(" WHERE "); + 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 "); + } + return sqlBuilder.toString(); + } + + public static void fillDeleteStatement(PreparedStatement st, IgniteMapping mapping, Object... data) throws SQLException { + assert mapping.getPrimaryKey().size() == data.length; + for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { + int j = i + 1; + st.setObject(j, data[i]); + } + } + + public static String selectGet(IgniteMapping mapping, List fields) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("SELECT "); + for (int i = 0; i < fields.size(); i++) { + sqlBuilder.append(fields.get(i)); + sqlBuilder.append(i == fields.size() - 1 ? "" : " , "); + } + sqlBuilder.append(" FROM "); + sqlBuilder.append(mapping.getTableName()); + sqlBuilder.append(" WHERE "); + 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 "); + } + return sqlBuilder.toString(); + } + + public static void fillSelectStatement(PreparedStatement st, IgniteMapping mapping, Object... data) throws SQLException { + assert mapping.getPrimaryKey().size() == data.length; + for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { + int j = i + 1; + st.setObject(j, data[i]); + } + } } diff --git a/gora-ignite/src/test/resources/gora-ignite-mapping.xml b/gora-ignite/src/test/resources/gora-ignite-mapping.xml index 3c1377463..6368e184e 100644 --- a/gora-ignite/src/test/resources/gora-ignite-mapping.xml +++ b/gora-ignite/src/test/resources/gora-ignite-mapping.xml @@ -19,17 +19,17 @@ - + - - + + - + @@ -37,6 +37,6 @@ - + From 87eddc2495cdeec9b28246ea08708540c365f34e Mon Sep 17 00:00:00 2001 From: Carlos M Date: Thu, 5 Jul 2018 23:11:38 -0500 Subject: [PATCH 5/8] Implement Queries for Ignite The operations newQuery, execute, deleteByQuery, getPartitions are implemented for the Ignite backend. The corresponding dependencies IgniteQuery and IgniteResult are also included. --- .../apache/gora/ignite/query/IgniteQuery.java | 32 ++++++ .../gora/ignite/query/IgniteResult.java | 101 ++++++++++++++++++ .../gora/ignite/query/package-info.java | 20 ++++ .../apache/gora/ignite/store/IgniteStore.java | 65 ++++++++++- .../gora/ignite/utils/IgniteSQLBuilder.java | 89 +++++++++++++++ .../gora/ignite/utils/package-info.java | 20 ++++ 6 files changed, 322 insertions(+), 5 deletions(-) create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/query/package-info.java create mode 100644 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/query/IgniteQuery.java b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java new file mode 100644 index 000000000..b33e6828b --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteQuery.java @@ -0,0 +1,32 @@ +/** + * 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 Query} interface. + */ +public class IgniteQuery extends QueryBase { + + public IgniteQuery(DataStore dataStore) { + super(dataStore); + } + +} 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 new file mode 100644 index 000000000..09f717e2c --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/IgniteResult.java @@ -0,0 +1,101 @@ +/** + * 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.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +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; + +/** + * IgniteResult specific implementation of the + * {@link org.apache.gora.query.Result} interface. + */ +public class IgniteResult extends ResultBase { + + private ResultSet resultSet; + private Statement st; + private int size; + + public IgniteResult(DataStore dataStore, Query query) { + super(dataStore, query); + } + + @Override + protected boolean nextInner() throws IOException { + try { + if (!resultSet.next()) { + return false; + } + key = ((IgniteStore) getDataStore()).extractKey(resultSet); + persistent = ((IgniteStore) getDataStore()).newInstance(resultSet, getQuery().getFields()); + return persistent != null; + } catch (SQLException ex) { + throw new IOException(ex); + } + } + + @Override + public float getProgress() throws IOException, InterruptedException { + if (resultSet == null) { + return 0; + } else if (size == 0) { + return 1; + } else { + return offset / (float) size; + } + } + + @Override + public int size() { + return size; + } + + @Override + public void close() throws IOException { + if (resultSet != null) { + try { + resultSet.close(); + } catch (SQLException ex) { + throw new IOException(ex); + } + } + if (st != null) { + try { + st.close(); + } catch (SQLException ex) { + throw new IOException(ex); + } + } + } + + public void setResultSet(ResultSet resultSet) throws SQLException { + this.resultSet = resultSet; + if (resultSet.last()) { + size = resultSet.getRow(); + } else { + size = 0; + } + resultSet.beforeFirst(); + } + +} 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 new file mode 100644 index 000000000..5e463ffe5 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/query/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 related all classes. + */ +package org.apache.gora.ignite.query; \ No newline at end of file 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 497e1612d..d9f352791 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 @@ -28,19 +28,24 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.Properties; import java.util.concurrent.ConcurrentHashMap; +import javax.sql.rowset.CachedRowSet; +import javax.sql.rowset.RowSetFactory; +import javax.sql.rowset.RowSetProvider; import org.apache.avro.Schema; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; import org.apache.avro.util.Utf8; +import org.apache.gora.ignite.query.IgniteQuery; +import org.apache.gora.ignite.query.IgniteResult; import org.apache.gora.ignite.utils.IgniteSQLBuilder; import org.apache.gora.persistency.Persistent; import org.apache.gora.persistency.impl.PersistentBase; import org.apache.gora.query.PartitionQuery; import org.apache.gora.query.Query; import org.apache.gora.query.Result; +import org.apache.gora.query.impl.PartitionQueryImpl; import org.apache.gora.store.impl.DataStoreBase; import org.apache.gora.util.AvroUtils; import org.apache.gora.util.GoraException; @@ -334,22 +339,72 @@ public boolean delete(K key) throws GoraException { @Override public long deleteByQuery(Query query) throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + + String deleteQuery; + if (query.getFields() != null && query.getFields().length < igniteMapping.getFields().size()) { + List dbFields = new ArrayList<>(); + for (String af : query.getFields()) { + dbFields.add(igniteMapping.getFields().get(af).getName()); + } + deleteQuery = IgniteSQLBuilder.deleteQueryFields(igniteMapping, dbFields); + } else { + deleteQuery = IgniteSQLBuilder.deleteQuery(igniteMapping); + } + String selectQueryWhere = IgniteSQLBuilder.selectQueryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit()); + try (PreparedStatement stmt = connection.prepareStatement(deleteQuery + selectQueryWhere)) { + IgniteSQLBuilder.fillSelectQuery(stmt, query.getStartKey(), query.getEndKey()); + stmt.executeUpdate(); + return 0; + } catch (SQLException ex) { + throw new GoraException(ex); + } } @Override public Result execute(Query query) throws GoraException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + String[] fields = getFieldsToQuery(query.getFields()); + //Avro fields to Ignite fields + List dbFields = new ArrayList<>(); + for (String af : fields) { + dbFields.add(igniteMapping.getFields().get(af).getName()); + } + String selectQuery = IgniteSQLBuilder.selectQuery(igniteMapping, dbFields); + String selectQueryWhere = IgniteSQLBuilder.selectQueryWhere(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()); + ResultSet executeQuery = stmt.executeQuery(); + rowset.populate(executeQuery); + IgniteResult igniteResult = new IgniteResult<>(this, query); + igniteResult.setResultSet(rowset); + return igniteResult; + } catch (SQLException ex) { + throw new GoraException(ex); + } + } + + public K extractKey(ResultSet r) throws SQLException { + assert igniteMapping.getPrimaryKey().size() == 1; + return (K) r.getObject(igniteMapping.getPrimaryKey().get(0).getName()); } @Override public Query newQuery() { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + IgniteQuery query = new IgniteQuery<>(this); + query.setFields(getFieldsToQuery(null)); + return query; } @Override public List> getPartitions(Query query) throws IOException { - throw new UnsupportedOperationException("Not supported yet."); //To change body of generated methods, choose Tools | Templates. + List> partitions = new ArrayList<>(); + PartitionQueryImpl partitionQuery = new PartitionQueryImpl<>( + query); + partitionQuery.setConf(getConf()); + partitions.add(partitionQuery); + return partitions; } @Override 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 92014c190..4c1e3143f 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 @@ -146,4 +146,93 @@ public static void fillSelectStatement(PreparedStatement st, IgniteMapping mappi st.setObject(j, data[i]); } } + + public static String selectQuery(IgniteMapping mapping, List ifields) { + List fields = new ArrayList<>(); + for (Column c : mapping.getPrimaryKey()) { + fields.add(c.getName()); + } + fields.addAll(ifields); + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("SELECT "); + for (int i = 0; i < fields.size(); i++) { + sqlBuilder.append(fields.get(i)); + sqlBuilder.append(i == fields.size() - 1 ? "" : " , "); + } + sqlBuilder.append(" FROM "); + sqlBuilder.append(mapping.getTableName()); + return sqlBuilder.toString(); + } + + public static String deleteQuery(IgniteMapping mapping) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("DELETE FROM "); + sqlBuilder.append(mapping.getTableName()); + return sqlBuilder.toString(); + } + + public static String deleteQueryFields(IgniteMapping mapping, List lsFields) { + StringBuilder sqlBuilder = new StringBuilder(); + sqlBuilder.append("UPDATE "); + sqlBuilder.append(mapping.getTableName()); + if (!lsFields.isEmpty()) { + sqlBuilder.append(" SET "); + } + for (int i = 0; i < lsFields.size(); i++) { + sqlBuilder.append(lsFields.get(i)); + sqlBuilder.append(" = null"); + sqlBuilder.append(i == lsFields.size() - 1 ? "" : " , "); + } + return sqlBuilder.toString(); + } + + public static String selectQueryWhere(IgniteMapping mapping, Object sk, Object ek, long limit) { + //composite keys pending + assert mapping.getPrimaryKey().size() == 1; + String keycolumn = mapping.getPrimaryKey().get(0).getName(); + StringBuilder sqlBuilder = new StringBuilder(); + if (sk != null || ek != null) { + sqlBuilder.append(" WHERE "); + if (sk != null && ek != null && sk.equals(ek)) { + sqlBuilder.append(keycolumn); + sqlBuilder.append("= ?"); + } else { + if (sk != null) { + sqlBuilder.append(keycolumn); + sqlBuilder.append(">= ?"); + } + if (sk != null && ek != null) { + sqlBuilder.append(" AND "); + } + if (ek != null) { + sqlBuilder.append(keycolumn); + sqlBuilder.append("<= ?"); + } + } + } + if (limit > 0) { + sqlBuilder.append(" LIMIT " + limit); + } + return sqlBuilder.toString(); + } + + public static void fillSelectQuery(PreparedStatement st, Object sk, Object ek) throws SQLException { + if (sk != null || ek != null) { + if (sk != null && ek != null && sk.equals(ek)) { + st.setObject(1, sk); + } else { + if (sk != null && ek != null) { + st.setObject(1, sk); + st.setObject(2, ek); + } else { + if (sk != null) { + st.setObject(1, sk); + } else { + st.setObject(1, ek); + } + } + } + } + } + } 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 new file mode 100644 index 000000000..ce4cd7aaf --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/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 related all classes. + */ +package org.apache.gora.ignite.utils; \ No newline at end of file From 740ab8145ac6c004d2b10ea46ea0f5c2a0a9f71f Mon Sep 17 00:00:00 2001 From: Carlos M Date: Thu, 26 Jul 2018 23:57:32 -0500 Subject: [PATCH 6/8] Improve Ignite Datastore to better suit Gora's coding standards Formatting and documentation were improved in multiple classes. Also dependencies' versions were parameterized within parent pom.xml files and other minor improvements were implemented throughout the project. --- .gitignore | 1 + gora-ignite/pom.xml | 4 +- gora-ignite/src/examples/java/.gitignore | 15 -- .../org/apache/gora/ignite/package-info.java | 4 +- .../apache/gora/ignite/query/IgniteQuery.java | 2 +- .../gora/ignite/query/IgniteResult.java | 10 +- .../gora/ignite/query/package-info.java | 2 +- .../org/apache/gora/ignite/store/Column.java | 2 +- .../gora/ignite/store/IgniteMapping.java | 2 +- .../ignite/store/IgniteMappingBuilder.java | 5 +- .../gora/ignite/store/IgniteParameters.java | 23 ++- .../apache/gora/ignite/store/IgniteStore.java | 59 +++--- .../gora/ignite/store/package-info.java | 2 +- .../ignite/utils/IgniteBackendConstants.java | 36 ++++ .../gora/ignite/utils/IgniteSQLBuilder.java | 193 ++++++++++++++---- .../gora/ignite/utils/package-info.java | 2 +- .../config/java.util.logging.properties | 74 +++++++ pom.xml | 2 + 18 files changed, 316 insertions(+), 122 deletions(-) delete mode 100644 gora-ignite/src/examples/java/.gitignore create mode 100644 gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java create mode 100644 gora-ignite/src/test/resources/META-INF/config/java.util.logging.properties diff --git a/.gitignore b/.gitignore index 2aa54cc30..d4f78bc64 100644 --- a/.gitignore +++ b/.gitignore @@ -37,3 +37,4 @@ 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 diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml index fa31195a8..430b8edbc 100644 --- a/gora-ignite/pom.xml +++ b/gora-ignite/pom.xml @@ -50,7 +50,6 @@ - 2.4.0 * org.apache.gora.ignite*;version="${project.version}";-noimport:=true @@ -174,9 +173,8 @@ org.apache.ignite ignite-indexing - 2.4.0 + ${ignite.version} - diff --git a/gora-ignite/src/examples/java/.gitignore b/gora-ignite/src/examples/java/.gitignore deleted file mode 100644 index 09697dce6..000000000 --- a/gora-ignite/src/examples/java/.gitignore +++ /dev/null @@ -1,15 +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. - diff --git a/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java b/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java index a7fa7aba0..403674811 100644 --- a/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/package-info.java @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -17,4 +17,4 @@ /** * This package contains Ignite datastore related all classes. */ -package org.apache.gora.ignite; \ No newline at end of file +package org.apache.gora.ignite; 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 b33e6828b..7ad9c6148 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 @@ -1,4 +1,4 @@ -/** +/* * 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. 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 09f717e2c..e5c0d4686 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 @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -33,7 +33,6 @@ public class IgniteResult extends ResultBase { private ResultSet resultSet; - private Statement st; private int size; public IgniteResult(DataStore dataStore, Query query) { @@ -79,13 +78,6 @@ public void close() throws IOException { throw new IOException(ex); } } - if (st != null) { - try { - st.close(); - } catch (SQLException ex) { - throw new IOException(ex); - } - } } public void setResultSet(ResultSet resultSet) throws SQLException { 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 5e463ffe5..fa52d9cd2 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 @@ -1,4 +1,4 @@ -/** +/* * 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. 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 d06591e5e..ee7d6d67e 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 @@ -1,4 +1,4 @@ -/** +/* * 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. 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 c0e7a98d5..f0189ec90 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 @@ -1,4 +1,4 @@ -/** +/* * 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. 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 3d245ea5a..4d397c5f1 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 @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -70,6 +70,7 @@ public void readMappingFile(String mappingFile) { LOG.error("Mapping file '{}' could not be found!", mappingFile); throw new IOException("Mapping file '" + mappingFile + "' could not be found!"); } + @SuppressWarnings("unchecked") List classes = document.getRootElement().getChildren("class"); for (Element classElement : classes) { if (classElement.getAttributeValue("keyClass").equals( @@ -79,6 +80,7 @@ public void readMappingFile(String mappingFile) { final String tableNameFromMapping = classElement.getAttributeValue("table"); String tableName = dataStore.getSchemaName(tableNameFromMapping, dataStore.getPersistentClass()); igniteMapping.setTableName(tableName); + @SuppressWarnings("unchecked") List prColumns = classElement.getChildren("primarykey"); List prFields = new ArrayList<>(); for (Element aPrimaryKey : prColumns) { @@ -87,6 +89,7 @@ public void readMappingFile(String mappingFile) { prFields.add(new Column(name, Column.FieldType.valueOf(type))); } igniteMapping.setPrimaryKey(prFields); + @SuppressWarnings("unchecked") List fields = classElement.getChildren("field"); Map mp = new HashMap<>(); for (Element field : fields) { 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 15b5baead..f667b40f4 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 @@ -1,11 +1,12 @@ /* - * Copyright 2018 The Apache Software Foundation. + * 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 * - * 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 + * 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, @@ -16,6 +17,7 @@ package org.apache.gora.ignite.store; import java.util.Properties; +import org.apache.gora.ignite.utils.IgniteBackendConstants; import org.apache.hadoop.conf.Configuration; /** @@ -61,8 +63,7 @@ public class IgniteParameters { private String additionalConfigurations; /** - * - * @param host + * @param host Hostname/IP of the Ignite Server * @param port Optional port for Ignite Server * @param user Optional username for Ignite * @param password Optional password for Ignite @@ -126,10 +127,10 @@ public void setSchema(String schema) { this.schema = schema; } - public static IgniteParameters load(Properties properties, Configuration conf) { + public static IgniteParameters load(Properties properties) { return new IgniteParameters( - properties.getProperty(PROP_HOST, "localhost"), - properties.getProperty(PROP_PORT, "10800"), + 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), 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 d9f352791..a17b3a0eb 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 @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -39,6 +39,7 @@ import org.apache.avro.util.Utf8; import org.apache.gora.ignite.query.IgniteQuery; import org.apache.gora.ignite.query.IgniteResult; +import org.apache.gora.ignite.utils.IgniteBackendConstants; import org.apache.gora.ignite.utils.IgniteSQLBuilder; import org.apache.gora.persistency.Persistent; import org.apache.gora.persistency.impl.PersistentBase; @@ -61,33 +62,24 @@ */ public class IgniteStore extends DataStoreBase { - public static final Logger LOG = LoggerFactory.getLogger(IgniteStore.class); + private static final Logger LOG = LoggerFactory.getLogger(IgniteStore.class); private static final String PARSE_MAPPING_FILE_KEY = "gora.ignite.mapping.file"; private static final String DEFAULT_MAPPING_FILE = "gora-ignite-mapping.xml"; private IgniteParameters igniteParameters; private IgniteMapping igniteMapping; private Connection connection; - - /* - * Create a threadlocal map for the datum readers and writers, because they - * are not thread safe, at least not before Avro 1.4.0 (See AVRO-650). When - * they are thread safe, it is possible to maintain a single reader and writer - * pair for every schema, instead of one for every thread. - */ - public static final ConcurrentHashMap> readerMap = new ConcurrentHashMap<>(); - - public static final ConcurrentHashMap> writerMap = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap> readerMap = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap> writerMap = new ConcurrentHashMap<>(); @Override public void initialize(Class keyClass, Class persistentClass, Properties properties) throws GoraException { - try { super.initialize(keyClass, persistentClass, properties); - IgniteMappingBuilder builder = new IgniteMappingBuilder(this); + IgniteMappingBuilder builder = new IgniteMappingBuilder(this); builder.readMappingFile(getConf().get(PARSE_MAPPING_FILE_KEY, DEFAULT_MAPPING_FILE)); igniteMapping = builder.getIgniteMapping(); - igniteParameters = IgniteParameters.load(properties, conf); - connection = acquiereConnection(); + igniteParameters = IgniteParameters.load(properties); + connection = acquireConnection(); LOG.info("Ignite store was successfully initialized"); } catch (ClassNotFoundException | SQLException ex) { LOG.error("Error while initializing Ignite store", ex); @@ -95,7 +87,7 @@ public void initialize(Class keyClass, Class persistentClass, Properties p } } - private Connection acquiereConnection() throws ClassNotFoundException, SQLException { + private Connection acquireConnection() throws ClassNotFoundException, SQLException { Class.forName("org.apache.ignite.IgniteJdbcThinDriver"); StringBuilder urlBuilder = new StringBuilder(); urlBuilder.append("jdbc:ignite:thin://"); @@ -115,8 +107,7 @@ private Connection acquiereConnection() throws ClassNotFoundException, SQLExcept if (igniteParameters.getAdditionalConfigurations() != null) { urlBuilder.append(igniteParameters.getAdditionalConfigurations()); } - Connection conn = DriverManager.getConnection(urlBuilder.toString()); - return conn; + return DriverManager.getConnection(urlBuilder.toString()); } @Override @@ -167,25 +158,19 @@ public void deleteSchema() throws GoraException { @Override public boolean schemaExists() throws GoraException { - boolean exists = false; try (Statement stmt = connection.createStatement()) { String tableExistsSQL = IgniteSQLBuilder.tableExists(igniteMapping.getTableName()); ResultSet executeQuery = stmt.executeQuery(tableExistsSQL); executeQuery.close(); - exists = true; + return true; } catch (SQLException ex) { - /** - * a 42000 error code is thrown by Ignite when a non-existent table - * queried. More details: - * https://apacheignite-sql.readme.io/docs/jdbc-error-codes - */ - if (ex.getSQLState() != null && ex.getSQLState().equals("42000")) { - exists = false; + if (ex.getSQLState() != null + && ex.getSQLState().equals(IgniteBackendConstants.DEFAULT_IGNITE_TABLE_NOT_EXISTS_CODE)) { + return false; } else { throw new GoraException(ex); } } - return exists; } @Override @@ -240,6 +225,7 @@ public T newInstance(ResultSet rs, String[] fields) throws GoraException, SQLExc return persistent; } + @SuppressWarnings("unchecked") private Object deserializeFieldValue(Schema.Field field, Schema fieldSchema, Object igniteValue, T persistent) throws IOException { Object fieldValue = null; @@ -298,7 +284,7 @@ public void put(K key, T obj) throws GoraException { Object v = obj.get(field.pos()); if (get != null && v != null) { Schema fieldSchema = field.schema(); - Object serializedObj = serializeFieldValue(get, fieldSchema, v); + Object serializedObj = serializeFieldValue(fieldSchema, v); data.put(get, serializedObj); } } @@ -350,7 +336,7 @@ public long deleteByQuery(Query query) throws GoraException { } else { deleteQuery = IgniteSQLBuilder.deleteQuery(igniteMapping); } - String selectQueryWhere = IgniteSQLBuilder.selectQueryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit()); + String selectQueryWhere = IgniteSQLBuilder.queryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit()); try (PreparedStatement stmt = connection.prepareStatement(deleteQuery + selectQueryWhere)) { IgniteSQLBuilder.fillSelectQuery(stmt, query.getStartKey(), query.getEndKey()); stmt.executeUpdate(); @@ -369,9 +355,8 @@ public Result execute(Query query) throws GoraException { dbFields.add(igniteMapping.getFields().get(af).getName()); } String selectQuery = IgniteSQLBuilder.selectQuery(igniteMapping, dbFields); - String selectQueryWhere = IgniteSQLBuilder.selectQueryWhere(igniteMapping, query.getStartKey(), query.getEndKey(), query.getLimit()); - try { - PreparedStatement stmt = connection.prepareStatement(selectQuery + selectQueryWhere); + String selectQueryWhere = IgniteSQLBuilder.queryWhere(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()); @@ -385,6 +370,7 @@ public Result execute(Query query) throws GoraException { } } + @SuppressWarnings("unchecked") public K extractKey(ResultSet r) throws SQLException { assert igniteMapping.getPrimaryKey().size() == 1; return (K) r.getObject(igniteMapping.getPrimaryKey().get(0).getName()); @@ -422,7 +408,8 @@ public void close() { } } - private Object serializeFieldValue(Column get, Schema fieldSchema, Object fieldValue) { + @SuppressWarnings("unchecked") + private Object serializeFieldValue(Schema fieldSchema, Object fieldValue) { Object output = fieldValue; switch (fieldSchema.getType()) { case ARRAY: @@ -442,7 +429,7 @@ private Object serializeFieldValue(Column get, Schema fieldSchema, Object fieldV if (fieldSchema.getTypes().size() == 2 && isNullable(fieldSchema)) { int schemaPos = getUnionSchema(fieldValue, fieldSchema); Schema unionSchema = fieldSchema.getTypes().get(schemaPos); - output = serializeFieldValue(get, unionSchema, fieldValue); + output = serializeFieldValue(unionSchema, fieldValue); } else { data = null; try { 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 62cba6785..49959c8c3 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 @@ -1,4 +1,4 @@ -/** +/* * 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. 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 new file mode 100644 index 000000000..7459c6716 --- /dev/null +++ b/gora-ignite/src/main/java/org/apache/gora/ignite/utils/IgniteBackendConstants.java @@ -0,0 +1,36 @@ +/* + * 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.utils; + +/** + * + * Constants file for Ignite. + */ +public class IgniteBackendConstants { + + /* + * 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"; +} 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 4c1e3143f..3639b90fd 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 @@ -1,4 +1,4 @@ -/** +/* * 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. @@ -25,7 +25,6 @@ import java.util.Locale; import java.util.Map; import java.util.Map.Entry; -import java.util.Set; import org.apache.gora.ignite.store.Column; import org.apache.gora.ignite.store.IgniteMapping; @@ -40,10 +39,23 @@ private static String format(String pattern, Object... args) { return messageFormat.format(args); } + /** + * Returns a SQL query for determine whether a table exists or not. + * + * @param tableName The name of the table to be check. + * @return SQL query + */ public static String tableExists(String tableName) { return format("SELECT * FROM {0} LIMIT 0", tableName); } + /** + * Returns a SQL create table statement for initializing a datastore based + * upon a Ignite Mapping definition. + * + * @param mapping The ignite mapping definition of the data store + * @return SQL create query (DDL). + */ public static String createTable(IgniteMapping mapping) { StringBuilder sqlBuilder = new StringBuilder(); sqlBuilder.append("CREATE TABLE "); @@ -67,10 +79,25 @@ public static String createTable(IgniteMapping mapping) { return sqlBuilder.toString(); } + /** + * Returns a SQL drop table statement for deleting a datastore instance within + * ignite. + * + * @param tableName The name of the table to be dropped. + * @return SQL drop query (DDL). + */ public static String dropTable(String tableName) { return format("DROP TABLE IF EXISTS {0} ;", tableName); } + /** + * Returns a bare SQL insert statement for adding a new record on a Ignite + * data store. + * + * @param mapping The ignite mapping definition of the data store + * @param data A map containing the Column-Value pairs of the new record. + * @return SQL insert statement + */ public static String baseInsertStatement(IgniteMapping mapping, Map data) { StringBuilder sqlBuilder = new StringBuilder(); sqlBuilder.append("MERGE INTO "); @@ -92,14 +119,31 @@ public static String baseInsertStatement(IgniteMapping mapping, Map data) throws SQLException { - List> list = new ArrayList<>(data.entrySet()); + /** + * Fills a SQL PreparedStatement of a insert operation with the actual data to + * be inserted. + * + * @param statement The insertion PreparedStatement to be filled. + * @param insertData A map containing the Column-Value pairs of the new + * record. + * @throws SQLException When invalid values are provided as parameters for the + * insert statement. + */ + public static void fillInsertStatement(PreparedStatement statement, Map insertData) throws SQLException { + List> list = new ArrayList<>(insertData.entrySet()); for (int i = 0; i < list.size(); i++) { int j = i + 1; - st.setObject(j, list.get(i).getValue()); + statement.setObject(j, list.get(i).getValue()); } } + /** + * Returns a bare SQL statement for deleting a record from the Ignite data + * store. + * + * @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 "); @@ -113,20 +157,38 @@ public static String delete(IgniteMapping mapping) { return sqlBuilder.toString(); } - public static void fillDeleteStatement(PreparedStatement st, IgniteMapping mapping, Object... data) throws SQLException { - assert mapping.getPrimaryKey().size() == data.length; + /** + * Fills a SQL PreparedStatement of a delete operation with the actual key of + * the record to be deleted + * + * @param statement The deletion PreparedStatement to be filled. + * @param mapping The ignite mapping definition of the data store + * @param deleteData An Object array containing the primary key values of the + * 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 { + assert mapping.getPrimaryKey().size() == deleteData.length; for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { int j = i + 1; - st.setObject(j, data[i]); + statement.setObject(j, deleteData[i]); } } - public static String selectGet(IgniteMapping mapping, List fields) { + /** + * Returns a bare SQL statement for retrieving a record from the ignite data + * store + * + * @param mapping The ignite mapping definition of the data store + * @param columns A list of columns to be retrieved within the select query + * @return SQL select statement + */ + public static String selectGet(IgniteMapping mapping, List columns) { StringBuilder sqlBuilder = new StringBuilder(); sqlBuilder.append("SELECT "); - for (int i = 0; i < fields.size(); i++) { - sqlBuilder.append(fields.get(i)); - sqlBuilder.append(i == fields.size() - 1 ? "" : " , "); + for (int i = 0; i < columns.size(); i++) { + sqlBuilder.append(columns.get(i)); + sqlBuilder.append(i == columns.size() - 1 ? "" : " , "); } sqlBuilder.append(" FROM "); sqlBuilder.append(mapping.getTableName()); @@ -139,20 +201,39 @@ public static String selectGet(IgniteMapping mapping, List fields) { return sqlBuilder.toString(); } - public static void fillSelectStatement(PreparedStatement st, IgniteMapping mapping, Object... data) throws SQLException { - assert mapping.getPrimaryKey().size() == data.length; + /** + * Fills a SQL PreparedStatement of a select operation with the actual keys of + * the record to be retrieved + * + * @param statement The select PreparedStatement to be filled. + * @param mapping The ignite mapping definition of the data store + * @param selectData An Object array containing the primary key values of the + * 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 { + assert mapping.getPrimaryKey().size() == selectData.length; for (int i = 0; i < mapping.getPrimaryKey().size(); i++) { int j = i + 1; - st.setObject(j, data[i]); + statement.setObject(j, selectData[i]); } } - public static String selectQuery(IgniteMapping mapping, List ifields) { + /** + * Returns a base SQL statement for retrieving multiple records from the + * ignite data store + * + * @param mapping The ignite mapping definition of the data store + * @param selectFields A list of columns to be retrieved within the select + * query + * @return SQL select statement + */ + public static String selectQuery(IgniteMapping mapping, List selectFields) { List fields = new ArrayList<>(); for (Column c : mapping.getPrimaryKey()) { fields.add(c.getName()); } - fields.addAll(ifields); + fields.addAll(selectFields); StringBuilder sqlBuilder = new StringBuilder(); sqlBuilder.append("SELECT "); for (int i = 0; i < fields.size(); i++) { @@ -164,6 +245,13 @@ public static String selectQuery(IgniteMapping mapping, List ifields) { return sqlBuilder.toString(); } + /** + * Returns a base SQL statement for deleting multiple records from the ignite + * data store + * + * @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 "); @@ -171,64 +259,91 @@ public static String deleteQuery(IgniteMapping mapping) { return sqlBuilder.toString(); } - public static String deleteQueryFields(IgniteMapping mapping, List lsFields) { + /** + * Returns a base SQL statement for deleting fields from records of the ignite + * data store + * + * @param mapping The ignite mapping definition of the data store + * @param deleteFields A list of columns to be deleted (set to null) + * @return SQL update statement + */ + public static String deleteQueryFields(IgniteMapping mapping, List deleteFields) { StringBuilder sqlBuilder = new StringBuilder(); sqlBuilder.append("UPDATE "); sqlBuilder.append(mapping.getTableName()); - if (!lsFields.isEmpty()) { + if (!deleteFields.isEmpty()) { sqlBuilder.append(" SET "); } - for (int i = 0; i < lsFields.size(); i++) { - sqlBuilder.append(lsFields.get(i)); + for (int i = 0; i < deleteFields.size(); i++) { + sqlBuilder.append(deleteFields.get(i)); sqlBuilder.append(" = null"); - sqlBuilder.append(i == lsFields.size() - 1 ? "" : " , "); + sqlBuilder.append(i == deleteFields.size() - 1 ? "" : " , "); } return sqlBuilder.toString(); } - public static String selectQueryWhere(IgniteMapping mapping, Object sk, Object ek, long limit) { + /** + * Returns a SQL's WHERE segment with proper conditions set for + * Querying/Deleting/Updating multiple records of a ignite data store + * + * @param mapping The ignite mapping definition of the data store + * @param startKey Start key of the WHERE condition + * @param endKey End key of the WHERE condition + * @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; String keycolumn = mapping.getPrimaryKey().get(0).getName(); StringBuilder sqlBuilder = new StringBuilder(); - if (sk != null || ek != null) { + if (startKey != null || endKey != null) { sqlBuilder.append(" WHERE "); - if (sk != null && ek != null && sk.equals(ek)) { + if (startKey != null && endKey != null && startKey.equals(endKey)) { sqlBuilder.append(keycolumn); sqlBuilder.append("= ?"); } else { - if (sk != null) { + if (startKey != null) { sqlBuilder.append(keycolumn); sqlBuilder.append(">= ?"); } - if (sk != null && ek != null) { + if (startKey != null && endKey != null) { sqlBuilder.append(" AND "); } - if (ek != null) { + if (endKey != null) { sqlBuilder.append(keycolumn); sqlBuilder.append("<= ?"); } } } if (limit > 0) { - sqlBuilder.append(" LIMIT " + limit); + sqlBuilder.append(" LIMIT ").append(limit); } return sqlBuilder.toString(); } - public static void fillSelectQuery(PreparedStatement st, Object sk, Object ek) throws SQLException { - if (sk != null || ek != null) { - if (sk != null && ek != null && sk.equals(ek)) { - st.setObject(1, sk); + /** + * Fills a SQL PreparedStatement's WHERE segment of a select/delete/update + * operation with proper key values + * + * @param statement The select PreparedStatement to be filled. + * @param startKey Start key of the WHERE condition + * @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 { + if (startKey != null || endKey != null) { + if (startKey != null && endKey != null && startKey.equals(endKey)) { + statement.setObject(1, startKey); } else { - if (sk != null && ek != null) { - st.setObject(1, sk); - st.setObject(2, ek); + if (startKey != null && endKey != null) { + statement.setObject(1, startKey); + statement.setObject(2, endKey); } else { - if (sk != null) { - st.setObject(1, sk); + if (startKey != null) { + statement.setObject(1, startKey); } else { - st.setObject(1, ek); + statement.setObject(1, endKey); } } } 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 ce4cd7aaf..ddba6e629 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 @@ -1,4 +1,4 @@ -/** +/* * 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. diff --git a/gora-ignite/src/test/resources/META-INF/config/java.util.logging.properties b/gora-ignite/src/test/resources/META-INF/config/java.util.logging.properties new file mode 100644 index 000000000..72e267f1f --- /dev/null +++ b/gora-ignite/src/test/resources/META-INF/config/java.util.logging.properties @@ -0,0 +1,74 @@ +# +# 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. +# + +######################################################################### +# Default java.util.logging configuration for Ignite. +# +# To use another config file use `java.util.logging.config.file` system +# property. For example `java -Djava.util.logging.config.file=myfile` +######################################################################### + +# +# Comma-separated list of logging "handlers". Note that some of them may be +# reconfigured (or even removed) at runtime according to system properties. +# +# By default all messages will be passed to console and file. +# +handlers=java.util.logging.ConsoleHandler, org.apache.ignite.logger.java.JavaLoggerFileHandler + +# +# Default global logging level. +# This specifies which kinds of events are logged across all loggers. +# For any given category this global level can be overriden by a category +# specific level. +# Note that handlers also have a separate level setting to limit messages +# printed through it. +# +.level=OFF + +# +# Uncomment to allow debug messages for entire Ignite package. +# +#org.apache.ignite.level=FINE + +# +# Uncomment this line to enable cache query execution tracing. +# +#org.apache.ignite.cache.queries.level=FINE + +# +# Uncomment to disable courtesy notices, such as SPI configuration +# consistency warnings. +# +#org.apache.ignite.CourtesyConfigNotice.level=OFF + +# +# Console handler logs all messages with importance level `INFO` and above +# into standard error stream (`System.err`). +# +java.util.logging.ConsoleHandler.formatter=org.apache.ignite.logger.java.JavaLoggerFormatter +java.util.logging.ConsoleHandler.level=INFO + +# +# File handler logs all messages into files with pattern `ignite-%{id8}.%g.log` +# under `$IGNITE_HOME/work/log/` directory. The placeholder `%{id8}` is a truncated node ID. +# +org.apache.ignite.logger.java.JavaLoggerFileHandler.formatter=org.apache.ignite.logger.java.JavaLoggerFormatter +org.apache.ignite.logger.java.JavaLoggerFileHandler.pattern=ignite-%{id8}.%g.log +org.apache.ignite.logger.java.JavaLoggerFileHandler.level=INFO +org.apache.ignite.logger.java.JavaLoggerFileHandler.limit=10485760 +org.apache.ignite.logger.java.JavaLoggerFileHandler.count=10 diff --git a/pom.xml b/pom.xml index 88b9b52da..b4838f0bf 100644 --- a/pom.xml +++ b/pom.xml @@ -780,6 +780,8 @@ 3.3.0 3.11.0 + + 2.6.0 6.5.1 6.5.1 From 8220a15915a86faa68419fe47fd78d4c349f8353 Mon Sep 17 00:00:00 2001 From: Carlos M Date: Sun, 29 Jul 2018 23:42:29 -0500 Subject: [PATCH 7/8] Add Ignite to the dependency management section The dependencies of the Ignite data store were added to the dependency management section. Also a unused dependency (Jline) was removed from the Ignite backend. --- gora-ignite/pom.xml | 8 -------- pom.xml | 23 ++++++++++++++++++++++- 2 files changed, 22 insertions(+), 9 deletions(-) diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml index 430b8edbc..8f7555305 100644 --- a/gora-ignite/pom.xml +++ b/gora-ignite/pom.xml @@ -118,14 +118,6 @@ commons-io - - - jline - jline - 2.11 - test - - org.apache.zookeeper zookeeper diff --git a/pom.xml b/pom.xml index b4838f0bf..de30ec635 100644 --- a/pom.xml +++ b/pom.xml @@ -956,6 +956,21 @@ ${project.version} test-jar + + + + + org.apache.gora + gora-ignite + ${project.version} + + + org.apache.gora + gora-ignite + ${project.version} + test-jar + + @@ -1573,7 +1588,13 @@ testcontainers ${test.container.version} - + + + + org.apache.ignite + ignite-indexing + ${ignite.version} + From 446f42ea3685691ee10cf5054a055e8f1685077f Mon Sep 17 00:00:00 2001 From: Carlos M Date: Sun, 9 Sep 2018 23:30:53 -0500 Subject: [PATCH 8/8] 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. --- .gitignore | 1 - gora-ignite/pom.xml | 9 +- .../apache/gora/ignite/query/IgniteQuery.java | 5 + .../gora/ignite/query/IgniteResult.java | 19 +- .../gora/ignite/query/package-info.java | 4 +- .../org/apache/gora/ignite/store/Column.java | 32 ++- .../gora/ignite/store/IgniteMapping.java | 33 +++ .../ignite/store/IgniteMappingBuilder.java | 22 +- .../gora/ignite/store/IgniteParameters.java | 111 ++++++---- .../apache/gora/ignite/store/IgniteStore.java | 67 +++--- .../gora/ignite/store/package-info.java | 4 +- .../ignite/utils/IgniteBackendConstants.java | 41 +++- .../gora/ignite/utils/IgniteSQLBuilder.java | 209 +++++++++--------- .../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 ++ .../gora/ignite/store/package-info.java | 20 ++ pom.xml | 117 +++++----- 19 files changed, 546 insertions(+), 255 deletions(-) create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/IgniteStoreMapReduceTest.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/mapreduce/package-info.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/package-info.java create mode 100644 gora-ignite/src/test/java/org/apache/gora/ignite/store/package-info.java diff --git a/.gitignore b/.gitignore index d4f78bc64..2aa54cc30 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 diff --git a/gora-ignite/pom.xml b/gora-ignite/pom.xml index 8f7555305..3e68e3b44 100644 --- a/gora-ignite/pom.xml +++ b/gora-ignite/pom.xml @@ -110,7 +110,6 @@ org.apache.ignite ignite-core - ${ignite.version} @@ -165,8 +164,12 @@ org.apache.ignite ignite-indexing - ${ignite.version} - + + com.healthmarketscience.sqlbuilder + sqlbuilder + + + 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 7ad9c6148..21715db6d 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 @@ 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 extends QueryBase { public IgniteQuery(DataStore dataStore) { super(dataStore); } + + public IgniteQuery() { + super(null); + } } 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 e5c0d4686..913e6880e 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 extends ResultBase private ResultSet resultSet; private int size; - public IgniteResult(DataStore dataStore, Query query) { + public IgniteResult(DataStore dataStore, Query 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 void close() throws IOException { } } - public void setResultSet(ResultSet resultSet) throws SQLException { - this.resultSet = resultSet; - if (resultSet.last()) { - size = resultSet.getRow(); - } else { - size = 0; - } - resultSet.beforeFirst(); - } - } 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 fa52d9cd2..0cbd9af4c 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; 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 ee7d6d67e..3034cdfe2 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 { 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 f0189ec90..4540ad499 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 fields; private List 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 getFields() { return fields; } + /** + * Sets field-column mappings + * + * @param fields Map containing mapped fields + */ public void setFields(Map fields) { this.fields = fields; } + /** + * Returns the primary key's list of columns + * + * @return List with columns + */ public List getPrimaryKey() { return primaryKey; } + /** + * Sets the primary key's columns + * + * @param primaryKey List with columns + */ public void setPrimaryKey(List primaryKey) { this.primaryKey = primaryKey; } 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 4d397c5f1..83ad3ba27 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 { private final IgniteStore dataStore; + /** + * Constructor for IgniteMappingBuilder + * + * @param store IgniteStore instance + */ public IgniteMappingBuilder(final IgniteStore 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 void readMappingFile(String mappingFile) { break; } } - } catch (IOException | JDOMException | ConfigurationException e) { throw new RuntimeException(e); } LOG.info("Gora Ignite mapping file was read successfully."); - } } 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 f667b40f4..66065835e 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 @@ 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 @@ private IgniteParameters(String host, String port, String schema, String user, S 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)); } } 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 a17b3a0eb..4f1abc95d 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 void initialize(Class keyClass, Class persistentClass, Properties p 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 void initialize(Class keyClass, Class persistentClass, Properties p } 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 T get(K key, String[] fields) throws GoraException { 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 T get(K key, String[] fields) throws GoraException { } 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 @@ private Object deserializeFieldValue(Schema.Field field, Schema fieldSchema, fieldValue = igniteValue; } return fieldValue; - } @Override @@ -280,17 +281,17 @@ public void put(K key, T obj) throws GoraException { //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 void put(K key, T obj) throws GoraException { @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 boolean delete(K key) throws GoraException { @Override public long deleteByQuery(Query query) throws GoraException { - String deleteQuery; if (query.getFields() != null && query.getFields().length < igniteMapping.getFields().size()) { List 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 Result execute(Query query) throws GoraException { String[] fields = getFieldsToQuery(query.getFields()); //Avro fields to Ignite fields List 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 igniteResult = new IgniteResult<>(this, query); - igniteResult.setResultSet(rowset); + IgniteResult igniteResult = new IgniteResult<>(this, query, rowset); return igniteResult; } catch (SQLException ex) { throw new GoraException(ex); @@ -462,7 +461,6 @@ private Object serializeFieldValue(Schema fieldSchema, Object fieldValue) { break; default: throw new AssertionError(fieldSchema.getType().name()); - } return output; } @@ -558,7 +556,6 @@ private SpecificDatumWriter getDatumWriter(Schema fieldSchema) { writer = new SpecificDatumWriter(fieldSchema);// ignore dirty bits writerMap.put(fieldSchema, writer); } - return writer; } 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 49959c8c3..e74747701 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; 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 7459c6716..9baa1316f 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 @@ */ 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"; + } 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 3639b90fd..85c296027 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 @@ */ 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 @@ private static String format(String pattern, Object... args) { * @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 static String tableExists(String tableName) { * @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 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 static String createTable(IgniteMapping mapping) { * @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 static String dropTable(String tableName) { * @param data A map containing the Column-Value pairs of the new record. * @return SQL insert statement */ - public static String baseInsertStatement(IgniteMapping mapping, Map data) { - StringBuilder sqlBuilder = new StringBuilder(); - sqlBuilder.append("MERGE INTO "); - sqlBuilder.append(mapping.getTableName()); - sqlBuilder.append(" ("); + public static String createInsertQuery(IgniteMapping mapping, Map data) { + DbSpec spec = new DbSpec(); + DbSchema schema = spec.addDefaultSchema(); + DbTable aTable = schema.addTable(mapping.getTableName()); + InsertQuery insertQuery = new InsertQuery(aTable); List> 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 static String baseInsertStatement(IgniteMapping mapping, Map insertData) throws SQLException { + public static void fillInsertQuery(PreparedStatement statement, Map insertData) throws SQLException { List> list = new ArrayList<>(insertData.entrySet()); for (int i = 0; i < list.size(); i++) { int j = i + 1; @@ -144,17 +149,17 @@ public static void fillInsertStatement(PreparedStatement statement, Map columns) { - StringBuilder sqlBuilder = new StringBuilder(); - sqlBuilder.append("SELECT "); + public static String createSelectQueryGet(IgniteMapping mapping, List 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 static String selectGet(IgniteMapping mapping, List columns) { * 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 static void fillSelectStatement(PreparedStatement statement, IgniteMappin * query * @return SQL select statement */ - public static String selectQuery(IgniteMapping mapping, List selectFields) { + public static String createSelectQuery(IgniteMapping mapping, List selectFields) { + DbSpec spec = new DbSpec(); + DbSchema schema = spec.addDefaultSchema(); + DbTable aTable = schema.addTable(mapping.getTableName()); + SelectQuery selectQuery = new SelectQuery(); + selectQuery.addFromTable(aTable); List 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 static String selectQuery(IgniteMapping mapping, List selectField * @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 static String deleteQuery(IgniteMapping mapping) { * @param deleteFields A list of columns to be deleted (set to null) * @return SQL update statement */ - public static String deleteQueryFields(IgniteMapping mapping, List 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 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 static String deleteQueryFields(IgniteMapping mapping, List delet * @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 static String queryWhere(IgniteMapping mapping, Object startKey, Object e * @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); 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 ddba6e629..78fa1d00b 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 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 000000000..2f82247e3 --- /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 createWebPageDataStore() throws IOException { + try { + return DataStoreFactory.getDataStore(String.class, WebPage.class, driver.getConfiguration()); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + +} 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 000000000..6dd4be708 --- /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; 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 000000000..1899e5de8 --- /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; 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 000000000..03fc7e352 --- /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; diff --git a/pom.xml b/pom.xml index de30ec635..ddd0d146e 100644 --- a/pom.xml +++ b/pom.xml @@ -433,7 +433,7 @@ + only. It has no influence on the Maven build itself. --> org.eclipse.m2e lifecycle-mapping @@ -647,17 +647,17 @@ + ${apache-rat-plugin.version} + rat-verify test check + + ASL20 Apache + Software License, 2.0 Single licensed ASL v2.0 + Licensed to the Apache Software Foundation (ASF) under + one or more contributor license agreements. + false CHANGES.txt + README.txt NOTICE.txt LICENSE.txt + KEYS doap_Gora.rdf .gitignore/**/** + org.apache.maven.plugins maven-source-plugin @@ -768,7 +768,7 @@ 4.2.0 1.8.1 - + 2.5.2 2.5.2 @@ -782,6 +782,7 @@ 3.11.0 2.6.0 + 2.1.7 6.5.1 6.5.1 @@ -970,6 +971,11 @@ ${project.version} test-jar + + com.healthmarketscience.sqlbuilder + sqlbuilder + ${sqlbuilder.version} + @@ -1045,26 +1051,26 @@ org.apache.hadoop avro - - slf4j-log4j12 - org.slf4j - - - hadoop-common - org.apache.hadoop - - - hadoop-yarn-common - org.apache.hadoop - - - hadoop-mapreduce-client-core - org.apache.hadoop - - - hadoop-auth - org.apache.hadoop - + + slf4j-log4j12 + org.slf4j + + + hadoop-common + org.apache.hadoop + + + hadoop-yarn-common + org.apache.hadoop + + + hadoop-mapreduce-client-core + org.apache.hadoop + + + hadoop-auth + org.apache.hadoop + @@ -1077,26 +1083,26 @@ org.apache.hadoop avro - - slf4j-log4j12 - org.slf4j - - - hadoop-common - org.apache.hadoop - - - hadoop-yarn-common - org.apache.hadoop - - - hadoop-mapreduce-client-core - org.apache.hadoop - - - hadoop-auth - org.apache.hadoop - + + slf4j-log4j12 + org.slf4j + + + hadoop-common + org.apache.hadoop + + + hadoop-yarn-common + org.apache.hadoop + + + hadoop-mapreduce-client-core + org.apache.hadoop + + + hadoop-auth + org.apache.hadoop + @@ -1595,6 +1601,11 @@ ignite-indexing ${ignite.version} + + org.apache.ignite + ignite-core + ${ignite.version} +