contactPointsWithPorts;
+
+ /** Maximum time to wait for schema agreement before returning from a DDL query. */
+ private Integer maxSchemaAgreementWaitSeconds;
+
+ /** The native protocol version to use. */
+ private Integer protoVer;
+
+ /** Compression to use for the transport. */
+ private String compression;
+
+ /** Use SSL for communications with Cassandra. */
+ private Boolean useSSL;
+
+ /** Enables metrics collection. */
+ private Boolean collectMetrix;
+
+ /** Enables JMX reporting of the metrics. */
+ private Boolean jmxReporting;
+
+ /** Credentials to use for authentication. */
+ private Credentials creds;
+
+ /** Load balancing policy to use. */
+ private LoadBalancingPolicy loadBalancingPlc;
+
+ /** Reconnection policy to use. */
+ private ReconnectionPolicy reconnectionPlc;
+
+ /** Retry policy to use. */
+ private RetryPolicy retryPlc;
+
+ /** Address translator to use. */
+ private AddressTranslator addrTranslator;
+
+ /** Speculative execution policy to use. */
+ private SpeculativeExecutionPolicy speculativeExecutionPlc;
+
+ /** Authentication provider to use. */
+ private AuthProvider authProvider;
+
+ /** SSL options to use. */
+ private SSLOptions sslOptions;
+
+ /** Connection pooling options to use. */
+ private PoolingOptions poolingOptions;
+
+ /** Socket options to use. */
+ private SocketOptions sockOptions;
+
+ /** Netty options to use for connection. */
+ private NettyOptions nettyOptions;
+
+ /** Expiration timeout for Cassandra driver session. */
+ private long sessionExpirationTimeout = DFLT_SESSION_EXPIRATION_TIMEOUT;
+
+ /** Cassandra session wrapper instance. */
+ private volatile CassandraSession ses;
+
+ /**
+ * Sets user name to use for authentication.
+ *
+ * @param user user name
+ */
+ public void setUser(String user) {
+ this.user = user;
+
+ invalidate();
+ }
+
+ /**
+ * Sets password to use for authentication.
+ *
+ * @param pwd password
+ */
+ public void setPassword(String pwd) {
+ this.pwd = pwd;
+
+ invalidate();
+ }
+
+ /**
+ * Sets port to use for Cassandra connection.
+ *
+ * @param port port
+ */
+ public void setPort(int port) {
+ this.port = port;
+
+ invalidate();
+ }
+
+ /**
+ * Sets list of contact points to connect to Cassandra cluster.
+ *
+ * @param points contact points
+ */
+ public void setContactPoints(String... points) {
+ if (points == null || points.length == 0)
+ return;
+
+ for (String point : points) {
+ if (point.contains(":")) {
+ if (contactPointsWithPorts == null)
+ contactPointsWithPorts = new LinkedList<>();
+
+ String[] chunks = point.split(":");
+
+ try {
+ contactPointsWithPorts.add(InetSocketAddress.createUnresolved(chunks[0].trim(), Integer.parseInt(chunks[1].trim())));
+ }
+ catch (Throwable e) {
+ throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
+ }
+ }
+ else {
+ if (contactPoints == null)
+ contactPoints = new LinkedList<>();
+
+ try {
+ contactPoints.add(InetAddress.getByName(point));
+ }
+ catch (Throwable e) {
+ throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
+ }
+ }
+ }
+
+ invalidate();
+ }
+
+ /** @param seconds Maximum time to wait for schema agreement before returning from a DDL query. */
+ public void setMaxSchemaAgreementWaitSeconds(int seconds) {
+ maxSchemaAgreementWaitSeconds = seconds;
+
+ invalidate();
+ }
+
+ /**
+ * Sets the native protocol version to use.
+ *
+ * @param ver version number
+ */
+ public void setProtocolVersion(int ver) {
+ protoVer = ver;
+
+ invalidate();
+ }
+
+ /**
+ * Sets compression algorithm to use for the transport.
+ *
+ * @param compression Compression algorithm.
+ */
+ public void setCompression(String compression) {
+ this.compression = compression == null || compression.trim().isEmpty() ? null : compression.trim();
+
+ try {
+ if (this.compression != null)
+ ProtocolOptions.Compression.valueOf(this.compression);
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Incorrect compression '" + compression + "' specified for Cassandra connection", e);
+ }
+
+ invalidate();
+ }
+
+ /**
+ * Enables SSL for communications with Cassandra.
+ *
+ * @param use Flag to enable/disable SSL.
+ */
+ public void setUseSSL(boolean use) {
+ useSSL = use;
+
+ invalidate();
+ }
+
+ /**
+ * Enables metrics collection.
+ *
+ * @param collect Flag to enable/disable metrics collection.
+ */
+ public void setCollectMetrix(boolean collect) {
+ collectMetrix = collect;
+
+ invalidate();
+ }
+
+ /**
+ * Enables JMX reporting of the metrics.
+ *
+ * @param enableReporting Flag to enable/disable JMX reporting.
+ */
+ public void setJmxReporting(boolean enableReporting) {
+ jmxReporting = enableReporting;
+
+ invalidate();
+ }
+
+ /**
+ * Sets number of rows to immediately fetch in CQL statement execution.
+ *
+ * @param size Number of rows to fetch.
+ */
+ public void setFetchSize(int size) {
+ fetchSize = size;
+
+ invalidate();
+ }
+
+ /**
+ * Set consistency level for READ operations.
+ *
+ * @param level Consistency level.
+ */
+ public void setReadConsistency(String level) {
+ readConsistency = parseConsistencyLevel(level);
+
+ invalidate();
+ }
+
+ /**
+ * Set consistency level for WRITE operations.
+ *
+ * @param level Consistency level.
+ */
+ public void setWriteConsistency(String level) {
+ writeConsistency = parseConsistencyLevel(level);
+
+ invalidate();
+ }
+
+ /**
+ * Sets credentials to use for authentication.
+ *
+ * @param creds Credentials.
+ */
+ public void setCredentials(Credentials creds) {
+ this.creds = creds;
+
+ invalidate();
+ }
+
+ /**
+ * Sets load balancing policy.
+ *
+ * @param plc Load balancing policy.
+ */
+ public void setLoadBalancingPolicy(LoadBalancingPolicy plc) {
+ loadBalancingPlc = plc;
+
+ invalidate();
+ }
+
+ /**
+ * Sets reconnection policy.
+ *
+ * @param plc Reconnection policy.
+ */
+ public void setReconnectionPolicy(ReconnectionPolicy plc) {
+ reconnectionPlc = plc;
+
+ invalidate();
+ }
+
+ /**
+ * Sets retry policy.
+ *
+ * @param plc Retry policy.
+ */
+ public void setRetryPolicy(RetryPolicy plc) {
+ retryPlc = plc;
+
+ invalidate();
+ }
+
+ /**
+ * Sets address translator.
+ *
+ * @param translator Address translator.
+ */
+ public void setAddressTranslator(AddressTranslator translator) {
+ addrTranslator = translator;
+
+ invalidate();
+ }
+
+ /**
+ * Sets speculative execution policy.
+ *
+ * @param plc Speculative execution policy.
+ */
+ public void setSpeculativeExecutionPolicy(SpeculativeExecutionPolicy plc) {
+ speculativeExecutionPlc = plc;
+
+ invalidate();
+ }
+
+ /**
+ * Sets authentication provider.
+ *
+ * @param provider Authentication provider.
+ */
+ public void setAuthProvider(AuthProvider provider) {
+ authProvider = provider;
+
+ invalidate();
+ }
+
+ /**
+ * Sets SSL options.
+ *
+ * @param options SSL options.
+ */
+ public void setSslOptions(SSLOptions options) {
+ sslOptions = options;
+
+ invalidate();
+ }
+
+ /**
+ * Sets pooling options.
+ *
+ * @param options pooling options to use.
+ */
+ public void setPoolingOptions(PoolingOptions options) {
+ poolingOptions = options;
+
+ invalidate();
+ }
+
+ /**
+ * Sets socket options to use.
+ *
+ * @param options Socket options.
+ */
+ public void setSocketOptions(SocketOptions options) {
+ sockOptions = options;
+
+ invalidate();
+ }
+
+ /**
+ * Sets netty options to use.
+ *
+ * @param options netty options.
+ */
+ public void setNettyOptions(NettyOptions options) {
+ nettyOptions = options;
+
+ invalidate();
+ }
+
+ /**
+ * Sets expiration timeout for Cassandra driver session. Idle sessions that are not
+ * used during this timeout value will be automatically closed and recreated later
+ * on demand.
+ *
+ * If set to {@code 0}, timeout is disabled.
+ *
+ * Default value is {@link #DFLT_SESSION_EXPIRATION_TIMEOUT}.
+ *
+ * @param sessionExpirationTimeout Expiration timeout for Cassandra driver session.
+ */
+ public void setSessionExpirationTimeout(long sessionExpirationTimeout) {
+ this.sessionExpirationTimeout = sessionExpirationTimeout;
+
+ invalidate();
+ }
+
+ /**
+ * Creates Cassandra session wrapper if it wasn't created yet and returns it
+ *
+ * @param log logger
+ * @return Cassandra session wrapper
+ */
+ public synchronized CassandraSession session(IgniteLogger log) {
+ if (ses != null)
+ return ses;
+
+ Cluster.Builder builder = Cluster.builder();
+
+ if (user != null)
+ builder = builder.withCredentials(user, pwd);
+
+ if (port != null)
+ builder = builder.withPort(port);
+
+ if (contactPoints != null)
+ builder = builder.addContactPoints(contactPoints);
+
+ if (contactPointsWithPorts != null)
+ builder = builder.addContactPointsWithPorts(contactPointsWithPorts);
+
+ if (maxSchemaAgreementWaitSeconds != null)
+ builder = builder.withMaxSchemaAgreementWaitSeconds(maxSchemaAgreementWaitSeconds);
+
+ if (protoVer != null)
+ builder = builder.withProtocolVersion(ProtocolVersion.fromInt(protoVer));
+
+ if (compression != null) {
+ try {
+ builder = builder.withCompression(ProtocolOptions.Compression.valueOf(compression.trim().toLowerCase()));
+ }
+ catch (IllegalArgumentException e) {
+ throw new IgniteException("Incorrect compression option '" + compression + "' specified for Cassandra connection", e);
+ }
+ }
+
+ if (useSSL != null && useSSL)
+ builder = builder.withSSL();
+
+ if (sslOptions != null)
+ builder = builder.withSSL(sslOptions);
+
+ if (collectMetrix != null && !collectMetrix)
+ builder = builder.withoutMetrics();
+
+ if (jmxReporting != null && !jmxReporting)
+ builder = builder.withoutJMXReporting();
+
+ if (creds != null)
+ builder = builder.withCredentials(creds.getUser(), creds.getPassword());
+
+ if (loadBalancingPlc != null)
+ builder = builder.withLoadBalancingPolicy(loadBalancingPlc);
+
+ if (reconnectionPlc != null)
+ builder = builder.withReconnectionPolicy(reconnectionPlc);
+
+ if (retryPlc != null)
+ builder = builder.withRetryPolicy(retryPlc);
+
+ if (addrTranslator != null)
+ builder = builder.withAddressTranslator(addrTranslator);
+
+ if (speculativeExecutionPlc != null)
+ builder = builder.withSpeculativeExecutionPolicy(speculativeExecutionPlc);
+
+ if (authProvider != null)
+ builder = builder.withAuthProvider(authProvider);
+
+ if (poolingOptions != null)
+ builder = builder.withPoolingOptions(poolingOptions);
+
+ if (sockOptions != null)
+ builder = builder.withSocketOptions(sockOptions);
+
+ if (nettyOptions != null)
+ builder = builder.withNettyOptions(nettyOptions);
+
+ return ses = new CassandraSessionImpl(
+ builder, fetchSize, readConsistency, writeConsistency, sessionExpirationTimeout, log);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void writeExternal(ObjectOutput out) throws IOException {
+ out.writeObject(fetchSize);
+ out.writeObject(readConsistency);
+ out.writeObject(writeConsistency);
+ U.writeString(out, user);
+ U.writeString(out, pwd);
+ out.writeObject(port);
+ out.writeObject(contactPoints);
+ out.writeObject(contactPointsWithPorts);
+ out.writeObject(maxSchemaAgreementWaitSeconds);
+ out.writeObject(protoVer);
+ U.writeString(out, compression);
+ out.writeObject(useSSL);
+ out.writeObject(collectMetrix);
+ out.writeObject(jmxReporting);
+ out.writeObject(creds);
+ writeObject(out, loadBalancingPlc);
+ writeObject(out, reconnectionPlc);
+ writeObject(out, addrTranslator);
+ writeObject(out, speculativeExecutionPlc);
+ writeObject(out, authProvider);
+ writeObject(out, sslOptions);
+ writeObject(out, poolingOptions);
+ writeObject(out, sockOptions);
+ writeObject(out, nettyOptions);
+ }
+
+ /** {@inheritDoc} */
+ @SuppressWarnings("unchecked")
+ @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+ fetchSize = (Integer)in.readObject();
+ readConsistency = (ConsistencyLevel)in.readObject();
+ writeConsistency = (ConsistencyLevel)in.readObject();
+ user = U.readString(in);
+ pwd = U.readString(in);
+ port = (Integer)in.readObject();
+ contactPoints = (List)in.readObject();
+ contactPointsWithPorts = (List)in.readObject();
+ maxSchemaAgreementWaitSeconds = (Integer)in.readObject();
+ protoVer = (Integer)in.readObject();
+ compression = U.readString(in);
+ useSSL = (Boolean)in.readObject();
+ collectMetrix = (Boolean)in.readObject();
+ jmxReporting = (Boolean)in.readObject();
+ creds = (Credentials)in.readObject();
+ loadBalancingPlc = (LoadBalancingPolicy)readObject(in);
+ reconnectionPlc = (ReconnectionPolicy)readObject(in);
+ addrTranslator = (AddressTranslator)readObject(in);
+ speculativeExecutionPlc = (SpeculativeExecutionPolicy)readObject(in);
+ authProvider = (AuthProvider)readObject(in);
+ sslOptions = (SSLOptions)readObject(in);
+ poolingOptions = (PoolingOptions)readObject(in);
+ sockOptions = (SocketOptions)readObject(in);
+ nettyOptions = (NettyOptions)readObject(in);
+ }
+
+ /**
+ * Helper method used to serialize class members
+ * @param out the stream to write the object to
+ * @param obj the object to be written
+ * @throws IOException Includes any I/O exceptions that may occur
+ */
+ private void writeObject(ObjectOutput out, Object obj) throws IOException {
+ out.writeObject(obj == null || !(obj instanceof Serializable) ? NULL_OBJECT : obj);
+ }
+
+ /**
+ * Helper method used to deserialize class members
+ * @param in the stream to read data from in order to restore the object
+ * @throws IOException Includes any I/O exceptions that may occur
+ * @throws ClassNotFoundException If the class for an object being restored cannot be found
+ * @return deserialized object
+ */
+ private Object readObject(ObjectInput in) throws IOException, ClassNotFoundException {
+ Object obj = in.readObject();
+ return NULL_OBJECT.equals(obj) ? null : obj;
+ }
+
+ /**
+ * Parses consistency level provided as string.
+ *
+ * @param level consistency level string.
+ *
+ * @return consistency level.
+ */
+ private ConsistencyLevel parseConsistencyLevel(String level) {
+ if (level == null)
+ return null;
+
+ try {
+ return ConsistencyLevel.valueOf(level.trim().toUpperCase());
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Incorrect consistency level '" + level + "' specified for Cassandra connection", e);
+ }
+ }
+
+ /**
+ * Invalidates session.
+ */
+ private synchronized void invalidate() {
+ ses = null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String toString() {
+ return S.toString(DataSource.class, this);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java
new file mode 100644
index 000000000..46ebdc543
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/PlainCredentials.java
@@ -0,0 +1,53 @@
+/*
+ * 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.ignite.cache.store.cassandra.datasource;
+
+/**
+ * Simple implementation of {@link Credentials} which just uses its constructor to hold user/password values.
+ */
+public class PlainCredentials implements Credentials {
+ /** */
+ private static final long serialVersionUID = 0L;
+
+ /** User name. */
+ private String user;
+
+ /** User password. */
+ private String pwd;
+
+ /**
+ * Creates credentials object.
+ *
+ * @param user User name.
+ * @param pwd User password.
+ */
+ public PlainCredentials(String user, String pwd) {
+ this.user = user;
+ this.pwd = pwd;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getUser() {
+ return user;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getPassword() {
+ return pwd;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java
new file mode 100644
index 000000000..6e1d22aed
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains data source implementation
+ */
+
+package org.apache.ignite.cache.store.cassandra.datasource;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java
new file mode 100644
index 000000000..00aee9040
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains {@link org.apache.ignite.cache.store.CacheStore} implementation backed by Cassandra database
+ */
+
+package org.apache.ignite.cache.store.cassandra;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java
new file mode 100644
index 000000000..dd4505811
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyPersistenceSettings.java
@@ -0,0 +1,249 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.ignite.cache.affinity.AffinityKeyMapped;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+/**
+ * Stores persistence settings for Ignite cache key
+ */
+public class KeyPersistenceSettings extends PersistenceSettings {
+ /** Partition key XML tag. */
+ private static final String PARTITION_KEY_ELEMENT = "partitionKey";
+
+ /** Cluster key XML tag. */
+ private static final String CLUSTER_KEY_ELEMENT = "clusterKey";
+
+ /** POJO field XML tag. */
+ private static final String FIELD_ELEMENT = "field";
+
+ /** POJO fields. */
+ private List fields = new LinkedList<>();
+
+ /** Partition key fields. */
+ private List partKeyFields = new LinkedList<>();
+
+ /** Cluster key fields. */
+ private List clusterKeyFields = new LinkedList<>();
+
+ /**
+ * Creates key persistence settings object based on it's XML configuration.
+ *
+ * @param el XML element storing key persistence settings
+ */
+ public KeyPersistenceSettings(Element el) {
+ super(el);
+
+ if (PersistenceStrategy.POJO != getStrategy()) {
+ init();
+
+ return;
+ }
+
+ Element node = el.getElementsByTagName(PARTITION_KEY_ELEMENT) != null ?
+ (Element)el.getElementsByTagName(PARTITION_KEY_ELEMENT).item(0) : null;
+
+ NodeList partKeysNodes = node == null ? null : node.getElementsByTagName(FIELD_ELEMENT);
+
+ node = el.getElementsByTagName(CLUSTER_KEY_ELEMENT) != null ?
+ (Element)el.getElementsByTagName(CLUSTER_KEY_ELEMENT).item(0) : null;
+
+ NodeList clusterKeysNodes = node == null ? null : node.getElementsByTagName(FIELD_ELEMENT);
+
+ if ((partKeysNodes == null || partKeysNodes.getLength() == 0) &&
+ clusterKeysNodes != null && clusterKeysNodes.getLength() > 0) {
+ throw new IllegalArgumentException("It's not allowed to specify cluster key fields mapping, but " +
+ "doesn't specify partition key mappings");
+ }
+
+ // Detecting partition key fields
+ partKeyFields = detectPojoFields(partKeysNodes);
+
+ if (partKeyFields == null || partKeyFields.isEmpty()) {
+ throw new IllegalStateException("Failed to initialize partition key fields for class '" +
+ getJavaClass().getName() + "'");
+ }
+
+ List filteredFields = new LinkedList<>();
+
+ // Find all fields annotated by @AffinityKeyMapped
+ for (PojoKeyField field : partKeyFields) {
+ if (field.getAnnotation(AffinityKeyMapped.class) != null)
+ filteredFields.add(field);
+ }
+
+ // If there are any fields annotated by @AffinityKeyMapped then all other fields are part of cluster key
+ partKeyFields = !filteredFields.isEmpty() ? filteredFields : partKeyFields;
+
+ // Detecting cluster key fields
+ clusterKeyFields = detectPojoFields(clusterKeysNodes);
+
+ filteredFields = new LinkedList<>();
+
+ // Removing out all fields which are already in partition key fields list
+ for (PojoKeyField field : clusterKeyFields) {
+ if (!PojoField.containsField(partKeyFields, field.getName()))
+ filteredFields.add(field);
+ }
+
+ clusterKeyFields = filteredFields;
+
+ fields = new LinkedList<>();
+ fields.addAll(partKeyFields);
+ fields.addAll(clusterKeyFields);
+
+ checkDuplicates(fields);
+
+ init();
+ }
+
+ /** {@inheritDoc} */
+ @Override public List getFields() {
+ return fields;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoKeyField createPojoField(Element el, Class clazz) {
+ return new PojoKeyField(el, clazz);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoKeyField createPojoField(PojoFieldAccessor accessor) {
+ return new PojoKeyField(accessor);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoKeyField createPojoField(PojoKeyField field, Class clazz) {
+ return new PojoKeyField(field, clazz);
+ }
+
+ /**
+ * Returns Cassandra DDL for primary key.
+ *
+ * @return DDL statement.
+ */
+ public String getPrimaryKeyDDL() {
+ StringBuilder partKey = new StringBuilder();
+
+ List cols = getPartitionKeyColumns();
+ for (String column : cols) {
+ if (partKey.length() != 0)
+ partKey.append(", ");
+
+ partKey.append("\"").append(column).append("\"");
+ }
+
+ StringBuilder clusterKey = new StringBuilder();
+
+ cols = getClusterKeyColumns();
+ if (cols != null) {
+ for (String column : cols) {
+ if (clusterKey.length() != 0)
+ clusterKey.append(", ");
+
+ clusterKey.append("\"").append(column).append("\"");
+ }
+ }
+
+ return clusterKey.length() == 0 ?
+ " primary key ((" + partKey + "))" :
+ " primary key ((" + partKey + "), " + clusterKey + ")";
+ }
+
+ /**
+ * Returns Cassandra DDL for cluster key.
+ *
+ * @return Cluster key DDL.
+ */
+ public String getClusteringDDL() {
+ StringBuilder builder = new StringBuilder();
+
+ for (PojoField field : clusterKeyFields) {
+ PojoKeyField.SortOrder sortOrder = ((PojoKeyField)field).getSortOrder();
+
+ if (sortOrder == null)
+ continue;
+
+ if (builder.length() != 0)
+ builder.append(", ");
+
+ boolean asc = PojoKeyField.SortOrder.ASC == sortOrder;
+
+ builder.append("\"").append(field.getColumn()).append("\" ").append(asc ? "asc" : "desc");
+ }
+
+ return builder.length() == 0 ? null : "clustering order by (" + builder + ")";
+ }
+
+ /** {@inheritDoc} */
+ @Override protected String defaultColumnName() {
+ return "key";
+ }
+
+ /**
+ * Returns partition key columns of Cassandra table.
+ *
+ * @return List of column names.
+ */
+ private List getPartitionKeyColumns() {
+ List cols = new LinkedList<>();
+
+ if (PersistenceStrategy.BLOB == getStrategy() || PersistenceStrategy.PRIMITIVE == getStrategy()) {
+ cols.add(getColumn());
+ return cols;
+ }
+
+ if (partKeyFields != null) {
+ for (PojoField field : partKeyFields)
+ cols.add(field.getColumn());
+ }
+
+ return cols;
+ }
+
+ /**
+ * Returns cluster key columns of Cassandra table.
+ *
+ * @return List of column names.
+ */
+ private List getClusterKeyColumns() {
+ List cols = new LinkedList<>();
+
+ if (clusterKeyFields != null) {
+ for (PojoField field : clusterKeyFields)
+ cols.add(field.getColumn());
+ }
+
+ return cols;
+ }
+
+ /**
+ * @see java.io.Serializable
+ */
+ private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+
+ fields = enrichFields(fields);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java
new file mode 100644
index 000000000..f865674e7
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/KeyValuePersistenceSettings.java
@@ -0,0 +1,531 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.Serializable;
+import java.io.StringReader;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+import org.apache.ignite.cache.store.cassandra.common.SystemHelper;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.springframework.core.io.Resource;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
+
+/**
+ * Stores persistence settings for Ignite cache key and value
+ */
+public class KeyValuePersistenceSettings implements Serializable {
+ /**
+ * Default Cassandra keyspace options which should be used to create new keyspace.
+ *
+ * SimpleStrategy for replication work well for single data center Cassandra cluster.
+ * If your Cassandra cluster deployed across multiple data centers it's better to use NetworkTopologyStrategy .
+ *
+ * Three replicas will be created for each data block.
+ * Setting DURABLE_WRITES to true specifies that all data should be written to commit log.
+ *
+ */
+ private static final String DFLT_KEYSPACE_OPTIONS = "replication = {'class' : 'SimpleStrategy', " +
+ "'replication_factor' : 3} and durable_writes = true";
+
+ /** Xml attribute specifying Cassandra keyspace to use. */
+ private static final String KEYSPACE_ATTR = "keyspace";
+
+ /** Xml attribute specifying Cassandra table to use. */
+ private static final String TABLE_ATTR = "table";
+
+ /** Xml attribute specifying ttl (time to leave) for rows inserted in Cassandra. */
+ private static final String TTL_ATTR = "ttl";
+
+ /** Root xml element containing persistence settings specification. */
+ private static final String PERSISTENCE_NODE = "persistence";
+
+ /** Xml element specifying Cassandra keyspace options. */
+ private static final String KEYSPACE_OPTIONS_NODE = "keyspaceOptions";
+
+ /** Xml element specifying Cassandra table options. */
+ private static final String TABLE_OPTIONS_NODE = "tableOptions";
+
+ /** Xml element specifying Ignite cache key persistence settings. */
+ private static final String KEY_PERSISTENCE_NODE = "keyPersistence";
+
+ /** Xml element specifying Ignite cache value persistence settings. */
+ private static final String VALUE_PERSISTENCE_NODE = "valuePersistence";
+
+ /**
+ * TTL (time to leave) for rows inserted into Cassandra table
+ * {@link Expiring data }.
+ */
+ private Integer ttl;
+
+ /** Cassandra keyspace (analog of tablespace in relational databases). */
+ private String keyspace;
+
+ /** Cassandra table. */
+ private String tbl;
+
+ /**
+ * Cassandra table creation options
+ * {@link CREATE TABLE }.
+ */
+ private String tblOptions;
+
+ /**
+ * Cassandra keyspace creation options
+ * {@link CREATE KEYSPACE }.
+ */
+ private String keyspaceOptions = DFLT_KEYSPACE_OPTIONS;
+
+ /** Persistence settings for Ignite cache keys. */
+ private KeyPersistenceSettings keyPersistenceSettings;
+
+ /** Persistence settings for Ignite cache values. */
+ private ValuePersistenceSettings valPersistenceSettings;
+
+ /** List of Cassandra table columns */
+ private List tableColumns;
+
+ /**
+ * Constructs Ignite cache key/value persistence settings.
+ *
+ * @param settings string containing xml with persistence settings for Ignite cache key/value
+ */
+ public KeyValuePersistenceSettings(String settings) {
+ init(settings);
+ }
+
+ /**
+ * Constructs Ignite cache key/value persistence settings.
+ *
+ * @param settingsFile xml file with persistence settings for Ignite cache key/value
+ */
+ public KeyValuePersistenceSettings(File settingsFile) {
+ InputStream in;
+
+ try {
+ in = new FileInputStream(settingsFile);
+ }
+ catch (IOException e) {
+ throw new IgniteException("Failed to get input stream for Cassandra persistence settings file: " +
+ settingsFile.getAbsolutePath(), e);
+ }
+
+ init(loadSettings(in));
+ }
+
+ /**
+ * Constructs Ignite cache key/value persistence settings.
+ *
+ * @param settingsRsrc resource containing xml with persistence settings for Ignite cache key/value
+ */
+ public KeyValuePersistenceSettings(Resource settingsRsrc) {
+ InputStream in;
+
+ try {
+ in = settingsRsrc.getInputStream();
+ }
+ catch (IOException e) {
+ throw new IgniteException("Failed to get input stream for Cassandra persistence settings resource: " + settingsRsrc, e);
+ }
+
+ init(loadSettings(in));
+ }
+
+ /**
+ * Returns ttl to use for while inserting new rows into Cassandra table.
+ *
+ * @return ttl
+ */
+ public Integer getTTL() {
+ return ttl;
+ }
+
+ /**
+ * Returns Cassandra keyspace to use.
+ *
+ * @return keyspace.
+ */
+ public String getKeyspace() {
+ return keyspace;
+ }
+
+ /**
+ * Returns Cassandra table to use.
+ *
+ * @return table.
+ */
+ public String getTable() {
+ return tbl;
+ }
+
+ /**
+ * Returns persistence settings for Ignite cache keys.
+ *
+ * @return keys persistence settings.
+ */
+ public KeyPersistenceSettings getKeyPersistenceSettings() {
+ return keyPersistenceSettings;
+ }
+
+ /**
+ * Returns persistence settings for Ignite cache values.
+ *
+ * @return values persistence settings.
+ */
+ public ValuePersistenceSettings getValuePersistenceSettings() {
+ return valPersistenceSettings;
+ }
+
+ /**
+ * Returns list of POJO fields to be mapped to Cassandra table columns.
+ *
+ * @return POJO fields list.
+ */
+ public List getFields() {
+ List fields = new LinkedList<>();
+
+ for (PojoField field : keyPersistenceSettings.getFields())
+ fields.add(field);
+
+ for (PojoField field : valPersistenceSettings.getFields())
+ fields.add(field);
+
+ return fields;
+ }
+
+ /**
+ * Returns list of Ignite cache key POJO fields to be mapped to Cassandra table columns.
+ *
+ * @return POJO fields list.
+ */
+ public List getKeyFields() {
+ return keyPersistenceSettings.getFields();
+ }
+
+ /**
+ * Returns list of Ignite cache value POJO fields to be mapped to Cassandra table columns.
+ *
+ * @return POJO fields list.
+ */
+ public List getValueFields() {
+ return valPersistenceSettings.getFields();
+ }
+
+ /**
+ * Returns DDL statement to create Cassandra keyspace.
+ *
+ * @return Keyspace DDL statement.
+ */
+ public String getKeyspaceDDLStatement() {
+ StringBuilder builder = new StringBuilder();
+ builder.append("create keyspace if not exists \"").append(keyspace).append("\"");
+
+ if (keyspaceOptions != null) {
+ if (!keyspaceOptions.trim().toLowerCase().startsWith("with"))
+ builder.append("\nwith");
+
+ builder.append(" ").append(keyspaceOptions);
+ }
+
+ String statement = builder.toString().trim().replaceAll(" +", " ");
+
+ return statement.endsWith(";") ? statement : statement + ";";
+ }
+
+ /**
+ * Returns column names for Cassandra table.
+ *
+ * @return Column names.
+ */
+ public List getTableColumns() {
+ return tableColumns;
+ }
+
+ /**
+ * Returns DDL statement to create Cassandra table.
+ *
+ * @param table Table name.
+ * @return Table DDL statement.
+ */
+ public String getTableDDLStatement(String table) {
+ if (table == null || table.trim().isEmpty())
+ throw new IllegalArgumentException("Table name should be specified");
+
+ String keyColumnsDDL = keyPersistenceSettings.getTableColumnsDDL();
+ String valColumnsDDL = valPersistenceSettings.getTableColumnsDDL(new HashSet<>(keyPersistenceSettings.getTableColumns()));
+
+ String colsDDL = keyColumnsDDL;
+
+ if (valColumnsDDL != null && !valColumnsDDL.trim().isEmpty())
+ colsDDL += ",\n" + valColumnsDDL;
+
+ String primaryKeyDDL = keyPersistenceSettings.getPrimaryKeyDDL();
+
+ String clusteringDDL = keyPersistenceSettings.getClusteringDDL();
+
+ String optionsDDL = tblOptions != null && !tblOptions.trim().isEmpty() ? tblOptions.trim() : "";
+
+ if (clusteringDDL != null && !clusteringDDL.isEmpty())
+ optionsDDL = optionsDDL.isEmpty() ? clusteringDDL : optionsDDL + " and " + clusteringDDL;
+
+ if (!optionsDDL.trim().isEmpty())
+ optionsDDL = optionsDDL.trim().toLowerCase().startsWith("with") ? optionsDDL.trim() : "with " + optionsDDL.trim();
+
+ StringBuilder builder = new StringBuilder();
+
+ builder.append("create table if not exists \"").append(keyspace).append("\".\"").append(table).append("\"");
+ builder.append("\n(\n").append(colsDDL).append(",\n").append(primaryKeyDDL).append("\n)");
+
+ if (!optionsDDL.isEmpty())
+ builder.append(" \n").append(optionsDDL);
+
+ String tblDDL = builder.toString().trim().replaceAll(" +", " ");
+
+ return tblDDL.endsWith(";") ? tblDDL : tblDDL + ";";
+ }
+
+ /**
+ * Returns DDL statements to create Cassandra table secondary indexes.
+ *
+ * @param table Table name.
+ * @return DDL statements to create secondary indexes.
+ */
+ public List getIndexDDLStatements(String table) {
+ List idxDDLs = new LinkedList<>();
+
+ Set keyCols = new HashSet<>(keyPersistenceSettings.getTableColumns());
+
+ List fields = valPersistenceSettings.getFields();
+
+ for (PojoField field : fields) {
+ if (!keyCols.contains(field.getColumn()) && ((PojoValueField)field).isIndexed())
+ idxDDLs.add(((PojoValueField)field).getIndexDDL(keyspace, table));
+ }
+
+ return idxDDLs;
+ }
+
+ /**
+ * Loads Ignite cache persistence settings from resource.
+ *
+ * @param in Input stream.
+ * @return String containing xml with Ignite cache persistence settings.
+ */
+ private String loadSettings(InputStream in) {
+ StringBuilder settings = new StringBuilder();
+ BufferedReader reader = null;
+
+ try {
+ reader = new BufferedReader(new InputStreamReader(in));
+
+ String line = reader.readLine();
+
+ while (line != null) {
+ if (settings.length() != 0)
+ settings.append(SystemHelper.LINE_SEPARATOR);
+
+ settings.append(line);
+
+ line = reader.readLine();
+ }
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to read input stream for Cassandra persistence settings", e);
+ }
+ finally {
+ U.closeQuiet(reader);
+ U.closeQuiet(in);
+ }
+
+ return settings.toString();
+ }
+
+ /**
+ * @param elem Element with data.
+ * @param attr Attribute name.
+ * @return Numeric value for specified attribute.
+ */
+ private int extractIntAttribute(Element elem, String attr) {
+ String val = elem.getAttribute(attr).trim();
+
+ try {
+ return Integer.parseInt(val);
+ }
+ catch (NumberFormatException ignored) {
+ throw new IllegalArgumentException("Incorrect value '" + val + "' specified for '" + attr + "' attribute");
+ }
+ }
+
+ /**
+ * Initializes persistence settings from XML string.
+ *
+ * @param settings XML string containing Ignite cache persistence settings configuration.
+ */
+ @SuppressWarnings("IfCanBeSwitch")
+ private void init(String settings) {
+ Document doc;
+
+ try {
+ DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+ DocumentBuilder builder = factory.newDocumentBuilder();
+ doc = builder.parse(new InputSource(new StringReader(settings)));
+ }
+ catch (Throwable e) {
+ throw new IllegalArgumentException("Failed to parse persistence settings:" +
+ SystemHelper.LINE_SEPARATOR + settings, e);
+ }
+
+ Element root = doc.getDocumentElement();
+
+ if (!PERSISTENCE_NODE.equals(root.getNodeName())) {
+ throw new IllegalArgumentException("Incorrect persistence settings specified. " +
+ "Root XML element should be 'persistence'");
+ }
+
+ if (!root.hasAttribute(KEYSPACE_ATTR)) {
+ throw new IllegalArgumentException("Incorrect persistence settings '" + KEYSPACE_ATTR +
+ "' attribute should be specified");
+ }
+
+ keyspace = root.getAttribute(KEYSPACE_ATTR).trim();
+ tbl = root.hasAttribute(TABLE_ATTR) ? root.getAttribute(TABLE_ATTR).trim() : null;
+
+ if (root.hasAttribute(TTL_ATTR))
+ ttl = extractIntAttribute(root, TTL_ATTR);
+
+ if (!root.hasChildNodes()) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+ "there are no key and value persistence settings specified");
+ }
+
+ NodeList children = root.getChildNodes();
+ int cnt = children.getLength();
+
+ for (int i = 0; i < cnt; i++) {
+ Node node = children.item(i);
+
+ if (node.getNodeType() != Node.ELEMENT_NODE)
+ continue;
+
+ Element el = (Element)node;
+ String nodeName = el.getNodeName();
+
+ if (nodeName.equals(TABLE_OPTIONS_NODE)) {
+ tblOptions = el.getTextContent();
+ tblOptions = tblOptions.replace("\n", " ").replace("\r", "").replace("\t", " ");
+ }
+ else if (nodeName.equals(KEYSPACE_OPTIONS_NODE)) {
+ keyspaceOptions = el.getTextContent();
+ keyspaceOptions = keyspaceOptions.replace("\n", " ").replace("\r", "").replace("\t", " ");
+ }
+ else if (nodeName.equals(KEY_PERSISTENCE_NODE))
+ keyPersistenceSettings = new KeyPersistenceSettings(el);
+ else if (nodeName.equals(VALUE_PERSISTENCE_NODE))
+ valPersistenceSettings = new ValuePersistenceSettings(el);
+ }
+
+ if (keyPersistenceSettings == null) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+ "there are no key persistence settings specified");
+ }
+
+ if (valPersistenceSettings == null) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+ "there are no value persistence settings specified");
+ }
+
+ List keyFields = keyPersistenceSettings.getFields();
+ List valFields = valPersistenceSettings.getFields();
+
+ if (PersistenceStrategy.POJO == keyPersistenceSettings.getStrategy() &&
+ (keyFields == null || keyFields.isEmpty())) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+ "there are no key fields found");
+ }
+
+ if (PersistenceStrategy.POJO == valPersistenceSettings.getStrategy() &&
+ (valFields == null || valFields.isEmpty())) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification, " +
+ "there are no value fields found");
+ }
+
+ // Validating aliases compatibility - fields having different names, but mapped to the same Cassandra table column.
+ if (valFields != null && !valFields.isEmpty()) {
+ String keyColumn = keyPersistenceSettings.getColumn();
+ Class keyClass = keyPersistenceSettings.getJavaClass();
+
+ if (keyColumn != null && !keyColumn.isEmpty()) {
+ for (PojoField valField : valFields) {
+ if (keyColumn.equals(valField.getColumn()) &&
+ !CassandraHelper.isCassandraCompatibleTypes(keyClass, valField.getJavaClass())) {
+ throw new IllegalArgumentException("Value field '" + valField.getName() + "' shares the same " +
+ "Cassandra table column '" + keyColumn + "' with key, but their Java classes are " +
+ "different. Fields sharing the same column should have the same Java class as their " +
+ "type or should be mapped to the same Cassandra primitive type.");
+ }
+ }
+ }
+
+ if (keyFields != null && !keyFields.isEmpty()) {
+ for (PojoField keyField : keyFields) {
+ for (PojoField valField : valFields) {
+ if (keyField.getColumn().equals(valField.getColumn()) &&
+ !CassandraHelper.isCassandraCompatibleTypes(keyField.getJavaClass(), valField.getJavaClass())) {
+ throw new IllegalArgumentException("Value field '" + valField.getName() + "' shares the same " +
+ "Cassandra table column '" + keyColumn + "' with key field '" + keyField.getName() + "', " +
+ "but their Java classes are different. Fields sharing the same column should have " +
+ "the same Java class as their type or should be mapped to the same Cassandra " +
+ "primitive type.");
+ }
+ }
+ }
+ }
+ }
+
+ tableColumns = new LinkedList<>();
+
+ for (String column : keyPersistenceSettings.getTableColumns()) {
+ if (!tableColumns.contains(column))
+ tableColumns.add(column);
+ }
+
+ for (String column : valPersistenceSettings.getTableColumns()) {
+ if (!tableColumns.contains(column))
+ tableColumns.add(column);
+ }
+
+ tableColumns = Collections.unmodifiableList(tableColumns);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java
new file mode 100644
index 000000000..59e066784
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceController.java
@@ -0,0 +1,459 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+
+/**
+ * Intermediate layer between persistent store (Cassandra) and Ignite cache key/value classes.
+ * Handles all the mappings to/from Java classes into Cassandra and responsible for all the details
+ * of how Java objects should be written/loaded to/from Cassandra.
+ */
+public class PersistenceController {
+ /** Ignite cache key/value persistence settings. */
+ private final KeyValuePersistenceSettings persistenceSettings;
+
+ /** List of key unique POJO fields (skipping aliases pointing to the same Cassandra table column). */
+ private final List extends PojoField> keyUniquePojoFields;
+
+ /** List of value unique POJO fields (skipping aliases pointing to the same Cassandra table column). */
+ private final List extends PojoField> valUniquePojoFields;
+
+ /** CQL statement template to insert row into Cassandra table. */
+ private final String writeStatementTempl;
+
+ /** CQL statement template to delete row from Cassandra table. */
+ private final String delStatementTempl;
+
+ /** CQL statement template to select value fields from Cassandra table. */
+ private final String loadStatementTempl;
+
+ /** CQL statement template to select key/value fields from Cassandra table. */
+ private final String loadWithKeyFieldsStatementTempl;
+
+ /** CQL statements to insert row into Cassandra table. */
+ private volatile Map writeStatements = new HashMap<>();
+
+ /** CQL statements to delete row from Cassandra table. */
+ private volatile Map delStatements = new HashMap<>();
+
+ /** CQL statements to select value fields from Cassandra table. */
+ private volatile Map loadStatements = new HashMap<>();
+
+ /** CQL statements to select key/value fields from Cassandra table. */
+ private volatile Map loadWithKeyFieldsStatements = new HashMap<>();
+
+ /**
+ * Constructs persistence controller from Ignite cache persistence settings.
+ *
+ * @param settings persistence settings.
+ */
+ public PersistenceController(KeyValuePersistenceSettings settings) {
+ if (settings == null)
+ throw new IllegalArgumentException("Persistent settings can't be null");
+
+ persistenceSettings = settings;
+
+ String[] loadStatements = prepareLoadStatements();
+
+ loadWithKeyFieldsStatementTempl = loadStatements[0];
+ loadStatementTempl = loadStatements[1];
+ writeStatementTempl = prepareWriteStatement();
+ delStatementTempl = prepareDeleteStatement();
+
+ keyUniquePojoFields = settings.getKeyPersistenceSettings().cassandraUniqueFields();
+
+ List extends PojoField> _valUniquePojoFields = settings.getValuePersistenceSettings().cassandraUniqueFields();
+
+ if (_valUniquePojoFields == null || _valUniquePojoFields.isEmpty()) {
+ valUniquePojoFields = _valUniquePojoFields;
+
+ return;
+ }
+
+ List keyColumns = new LinkedList<>();
+
+ if (keyUniquePojoFields == null)
+ keyColumns.add(settings.getKeyPersistenceSettings().getColumn());
+ else {
+ for (PojoField field : keyUniquePojoFields)
+ keyColumns.add(field.getColumn());
+ }
+
+ List fields = new LinkedList<>(_valUniquePojoFields);
+
+ for (String column : keyColumns) {
+ for (int i = 0; i < fields.size(); i++) {
+ if (column.equals(fields.get(i).getColumn())) {
+ fields.remove(i);
+ break;
+ }
+ }
+ }
+
+ valUniquePojoFields = fields.isEmpty() ? null : Collections.unmodifiableList(fields);
+ }
+
+ /**
+ * Returns Ignite cache persistence settings.
+ *
+ * @return persistence settings.
+ */
+ public KeyValuePersistenceSettings getPersistenceSettings() {
+ return persistenceSettings;
+ }
+
+ /**
+ * Returns CQL statement to insert row into Cassandra table.
+ *
+ * @param table Table name.
+ * @return CQL statement.
+ */
+ public String getWriteStatement(String table) {
+ return getStatement(table, writeStatementTempl, writeStatements);
+ }
+
+ /**
+ * Returns CQL statement to delete row from Cassandra table.
+ *
+ * @param table Table name.
+ * @return CQL statement.
+ */
+ public String getDeleteStatement(String table) {
+ return getStatement(table, delStatementTempl, delStatements);
+ }
+
+ /**
+ * Returns CQL statement to select key/value fields from Cassandra table.
+ *
+ * @param table Table name.
+ * @param includeKeyFields whether to include/exclude key fields from the returned row.
+ *
+ * @return CQL statement.
+ */
+ public String getLoadStatement(String table, boolean includeKeyFields) {
+ return includeKeyFields ?
+ getStatement(table, loadWithKeyFieldsStatementTempl, loadWithKeyFieldsStatements) :
+ getStatement(table, loadStatementTempl, loadStatements);
+ }
+
+ /**
+ * Binds Ignite cache key object to {@link PreparedStatement}.
+ *
+ * @param statement statement to which key object should be bind.
+ * @param key key object.
+ *
+ * @return statement with bounded key.
+ */
+ public BoundStatement bindKey(PreparedStatement statement, Object key) {
+ PersistenceSettings settings = persistenceSettings.getKeyPersistenceSettings();
+
+ Object[] values = PersistenceStrategy.POJO != settings.getStrategy() ?
+ new Object[1] : new Object[keyUniquePojoFields.size()];
+
+ bindValues(settings.getStrategy(), settings.getSerializer(), keyUniquePojoFields, key, values, 0);
+
+ return statement.bind(values);
+ }
+
+ /**
+ * Binds Ignite cache key and value object to {@link com.datastax.driver.core.PreparedStatement}.
+ *
+ * @param statement statement to which key and value object should be bind.
+ * @param key key object.
+ * @param val value object.
+ *
+ * @return statement with bounded key and value.
+ */
+ public BoundStatement bindKeyValue(PreparedStatement statement, Object key, Object val) {
+ Object[] values = new Object[persistenceSettings.getTableColumns().size()];
+
+ PersistenceSettings keySettings = persistenceSettings.getKeyPersistenceSettings();
+ PersistenceSettings valSettings = persistenceSettings.getValuePersistenceSettings();
+
+ int offset = bindValues(keySettings.getStrategy(), keySettings.getSerializer(), keyUniquePojoFields, key, values, 0);
+ bindValues(valSettings.getStrategy(), valSettings.getSerializer(), valUniquePojoFields, val, values, offset);
+
+ return statement.bind(values);
+ }
+
+ /**
+ * Builds Ignite cache key object from returned Cassandra table row.
+ *
+ * @param row Cassandra table row.
+ *
+ * @return key object.
+ */
+ public Object buildKeyObject(Row row) {
+ return buildObject(row, persistenceSettings.getKeyPersistenceSettings());
+ }
+
+ /**
+ * Builds Ignite cache value object from Cassandra table row .
+ *
+ * @param row Cassandra table row.
+ *
+ * @return value object.
+ */
+ public Object buildValueObject(Row row) {
+ return buildObject(row, persistenceSettings.getValuePersistenceSettings());
+ }
+
+ /**
+ * Service method to prepare CQL write statement.
+ *
+ * @return CQL write statement.
+ */
+ private String prepareWriteStatement() {
+ Collection cols = persistenceSettings.getTableColumns();
+
+ StringBuilder colsList = new StringBuilder();
+ StringBuilder questionsList = new StringBuilder();
+
+ for (String column : cols) {
+ if (colsList.length() != 0) {
+ colsList.append(", ");
+ questionsList.append(",");
+ }
+
+ colsList.append("\"").append(column).append("\"");
+ questionsList.append("?");
+ }
+
+ String statement = "insert into \"" + persistenceSettings.getKeyspace() + "\".\"%1$s" +
+ "\" (" + colsList + ") values (" + questionsList + ")";
+
+ if (persistenceSettings.getTTL() != null)
+ statement += " using ttl " + persistenceSettings.getTTL();
+
+ return statement + ";";
+ }
+
+ /**
+ * Service method to prepare CQL delete statement.
+ *
+ * @return CQL write statement.
+ */
+ private String prepareDeleteStatement() {
+ Collection cols = persistenceSettings.getKeyPersistenceSettings().getTableColumns();
+
+ StringBuilder statement = new StringBuilder();
+
+ for (String column : cols) {
+ if (statement.length() != 0)
+ statement.append(" and ");
+
+ statement.append("\"").append(column).append("\"=?");
+ }
+
+ statement.append(";");
+
+ return "delete from \"" + persistenceSettings.getKeyspace() + "\".\"%1$s\" where " + statement;
+ }
+
+ /**
+ * Service method to prepare CQL load statements including and excluding key columns.
+ *
+ * @return array having two CQL statements (including and excluding key columns).
+ */
+ private String[] prepareLoadStatements() {
+ PersistenceSettings settings = persistenceSettings.getKeyPersistenceSettings();
+ boolean pojoStrategy = PersistenceStrategy.POJO == settings.getStrategy();
+ Collection keyCols = settings.getTableColumns();
+ StringBuilder hdrWithKeyFields = new StringBuilder();
+
+ for (String column : keyCols) {
+ // omit calculated fields in load statement
+ if (pojoStrategy && settings.getFieldByColumn(column).calculatedField())
+ continue;
+
+ if (hdrWithKeyFields.length() > 0)
+ hdrWithKeyFields.append(", ");
+
+ hdrWithKeyFields.append("\"").append(column).append("\"");
+ }
+
+ settings = persistenceSettings.getValuePersistenceSettings();
+ pojoStrategy = PersistenceStrategy.POJO == settings.getStrategy();
+ Collection valCols = settings.getTableColumns();
+ StringBuilder hdr = new StringBuilder();
+
+ for (String column : valCols) {
+ // omit calculated fields in load statement
+ if (pojoStrategy && settings.getFieldByColumn(column).calculatedField())
+ continue;
+
+ if (hdr.length() > 0)
+ hdr.append(", ");
+
+ hdr.append("\"").append(column).append("\"");
+
+ if (!keyCols.contains(column))
+ hdrWithKeyFields.append(", \"").append(column).append("\"");
+ }
+
+ hdrWithKeyFields.insert(0, "select ");
+ hdr.insert(0, "select ");
+
+ StringBuilder statement = new StringBuilder();
+
+ statement.append(" from \"");
+ statement.append(persistenceSettings.getKeyspace());
+ statement.append("\".\"%1$s");
+ statement.append("\" where ");
+
+ int i = 0;
+
+ for (String column : keyCols) {
+ if (i > 0)
+ statement.append(" and ");
+
+ statement.append("\"").append(column).append("\"=?");
+ i++;
+ }
+
+ statement.append(";");
+
+ return new String[] {hdrWithKeyFields + statement.toString(), hdr + statement.toString()};
+ }
+
+ /**
+ * @param table Table.
+ * @param template Template.
+ * @param statements Statements.
+ * @return Statement.
+ */
+ private String getStatement(final String table, final String template, final Map statements) {
+ //noinspection SynchronizationOnLocalVariableOrMethodParameter
+ synchronized (statements) {
+ String st = statements.get(table);
+
+ if (st == null) {
+ st = String.format(template, table);
+ statements.put(table, st);
+ }
+
+ return st;
+ }
+ }
+
+ /**
+ * Builds object from Cassandra table row.
+ *
+ * @param row Cassandra table row.
+ * @param settings persistence settings to use.
+ *
+ * @return object.
+ */
+ private Object buildObject(Row row, PersistenceSettings settings) {
+ if (row == null)
+ return null;
+
+ PersistenceStrategy stg = settings.getStrategy();
+
+ Class clazz = settings.getJavaClass();
+ String col = settings.getColumn();
+
+ if (PersistenceStrategy.PRIMITIVE == stg)
+ return PropertyMappingHelper.getCassandraColumnValue(row, col, clazz, null);
+
+ if (PersistenceStrategy.BLOB == stg)
+ return settings.getSerializer().deserialize(row.getBytes(col));
+
+ List fields = settings.getFields();
+
+ Object obj;
+
+ try {
+ obj = clazz.newInstance();
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to instantiate object of type '" + clazz.getName() + "' using reflection", e);
+ }
+
+ for (PojoField field : fields) {
+ if (!field.calculatedField())
+ field.setValueFromRow(row, obj, settings.getSerializer());
+ }
+
+ return obj;
+ }
+
+ /**
+ * Extracts field values from POJO object, converts into Java types
+ * which could be mapped to Cassandra types and stores them inside provided values
+ * array starting from specified offset.
+ *
+ * @param stgy Persistence strategy to use.
+ * @param serializer Serializer to use for BLOBs.
+ * @param fields Fields who's values should be extracted.
+ * @param obj Object instance who's field values should be extracted.
+ * @param values Array to store values.
+ * @param offset Offset starting from which to store fields values in the provided values array.
+ *
+ * @return next offset
+ */
+ private int bindValues(PersistenceStrategy stgy, Serializer serializer, List extends PojoField> fields, Object obj,
+ Object[] values, int offset) {
+ if (PersistenceStrategy.PRIMITIVE == stgy) {
+ if (PropertyMappingHelper.getCassandraType(obj.getClass()) == null ||
+ obj.getClass().equals(ByteBuffer.class) || obj instanceof byte[]) {
+ throw new IllegalArgumentException("Couldn't deserialize instance of class '" +
+ obj.getClass().getName() + "' using PRIMITIVE strategy. Please use BLOB strategy for this case.");
+ }
+
+ values[offset] = obj;
+
+ return ++offset;
+ }
+
+ if (PersistenceStrategy.BLOB == stgy) {
+ values[offset] = serializer.serialize(obj);
+
+ return ++offset;
+ }
+
+ if (fields == null || fields.isEmpty())
+ return offset;
+
+ for (PojoField field : fields) {
+ Object val = field.getValueFromObject(obj, serializer);
+
+ if (val instanceof byte[])
+ val = ByteBuffer.wrap((byte[])val);
+
+ values[offset] = val;
+
+ offset++;
+ }
+
+ return offset;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java
new file mode 100644
index 000000000..6a0d703a3
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceSettings.java
@@ -0,0 +1,557 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import java.io.IOException;
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import com.datastax.driver.core.DataType;
+import org.apache.commons.beanutils.PropertyUtils;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+/**
+ * Stores persistence settings, which describes how particular key/value
+ * from Ignite cache should be stored in Cassandra.
+ */
+public abstract class PersistenceSettings implements Serializable {
+ /** Xml attribute specifying persistence strategy. */
+ private static final String STRATEGY_ATTR = "strategy";
+
+ /** Xml attribute specifying Cassandra column name. */
+ private static final String COLUMN_ATTR = "column";
+
+ /** Xml attribute specifying BLOB serializer to use. */
+ private static final String SERIALIZER_ATTR = "serializer";
+
+ /** Xml attribute specifying java class of the object to be persisted. */
+ private static final String CLASS_ATTR = "class";
+
+ /** Persistence strategy to use. */
+ private PersistenceStrategy stgy;
+
+ /** Java class of the object to be persisted. */
+ private Class javaCls;
+
+ /** Cassandra table column name where object should be persisted in
+ * case of using BLOB or PRIMITIVE persistence strategy. */
+ private String col;
+
+ /** Serializer for BLOBs. */
+ private Serializer serializer = new JavaSerializer();
+
+ /** List of Cassandra table columns */
+ private List tableColumns;
+
+ /**
+ * List of POJO fields having unique mapping to Cassandra columns - skipping aliases pointing
+ * to the same Cassandra table column.
+ */
+ private List casUniqueFields;
+
+ /**
+ * Extracts property descriptor from the descriptors list by its name.
+ *
+ * @param descriptors descriptors list.
+ * @param propName property name.
+ *
+ * @return property descriptor.
+ */
+ public static PropertyDescriptor findPropertyDescriptor(List descriptors, String propName) {
+ if (descriptors == null || descriptors.isEmpty() || propName == null || propName.trim().isEmpty())
+ return null;
+
+ for (PropertyDescriptor descriptor : descriptors) {
+ if (descriptor.getName().equals(propName))
+ return descriptor;
+ }
+
+ return null;
+ }
+
+ /**
+ * Constructs persistence settings from corresponding XML element.
+ *
+ * @param el xml element containing persistence settings configuration.
+ */
+ @SuppressWarnings("unchecked")
+ public PersistenceSettings(Element el) {
+ if (el == null)
+ throw new IllegalArgumentException("DOM element representing key/value persistence object can't be null");
+
+ if (!el.hasAttribute(STRATEGY_ATTR)) {
+ throw new IllegalArgumentException("DOM element representing key/value persistence object should have '" +
+ STRATEGY_ATTR + "' attribute");
+ }
+
+ try {
+ stgy = PersistenceStrategy.valueOf(el.getAttribute(STRATEGY_ATTR).trim().toUpperCase());
+ }
+ catch (IllegalArgumentException ignored) {
+ throw new IllegalArgumentException("Incorrect persistence strategy specified: " + el.getAttribute(STRATEGY_ATTR));
+ }
+
+ if (!el.hasAttribute(CLASS_ATTR) && PersistenceStrategy.BLOB != stgy) {
+ throw new IllegalArgumentException("DOM element representing key/value persistence object should have '" +
+ CLASS_ATTR + "' attribute or have BLOB persistence strategy");
+ }
+
+ try {
+ javaCls = el.hasAttribute(CLASS_ATTR) ? getClassInstance(el.getAttribute(CLASS_ATTR).trim()) : null;
+ }
+ catch (Throwable e) {
+ throw new IllegalArgumentException("Incorrect java class specified '" + el.getAttribute(CLASS_ATTR) + "' " +
+ "for Cassandra persistence", e);
+ }
+
+ if (PersistenceStrategy.BLOB != stgy &&
+ (ByteBuffer.class.equals(javaCls) || byte[].class.equals(javaCls))) {
+ throw new IllegalArgumentException("Java class '" + el.getAttribute(CLASS_ATTR) + "' " +
+ "specified could only be persisted using BLOB persistence strategy");
+ }
+
+ if (PersistenceStrategy.PRIMITIVE == stgy &&
+ PropertyMappingHelper.getCassandraType(javaCls) == null) {
+ throw new IllegalArgumentException("Current implementation doesn't support persisting '" +
+ javaCls.getName() + "' object using PRIMITIVE strategy");
+ }
+
+ if (PersistenceStrategy.POJO == stgy) {
+ if (javaCls == null)
+ throw new IllegalStateException("Object java class should be specified for POJO persistence strategy");
+
+ try {
+ javaCls.getConstructor();
+ }
+ catch (Throwable e) {
+ throw new IllegalArgumentException("Java class '" + javaCls.getName() + "' couldn't be used as POJO " +
+ "cause it doesn't have no arguments constructor", e);
+ }
+ }
+
+ if (el.hasAttribute(COLUMN_ATTR)) {
+ if (PersistenceStrategy.BLOB != stgy && PersistenceStrategy.PRIMITIVE != stgy) {
+ throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+ "'" + COLUMN_ATTR + "' attribute is only applicable for PRIMITIVE or BLOB strategy");
+ }
+
+ col = el.getAttribute(COLUMN_ATTR).trim();
+ }
+
+ if (el.hasAttribute(SERIALIZER_ATTR)) {
+ if (PersistenceStrategy.BLOB != stgy && PersistenceStrategy.POJO != stgy) {
+ throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+ "'" + SERIALIZER_ATTR + "' attribute is only applicable for BLOB and POJO strategies");
+ }
+
+ Object obj = newObjectInstance(el.getAttribute(SERIALIZER_ATTR).trim());
+
+ if (!(obj instanceof Serializer)) {
+ throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " +
+ "serializer class '" + el.getAttribute(SERIALIZER_ATTR) + "' doesn't implement '" +
+ Serializer.class.getName() + "' interface");
+ }
+
+ serializer = (Serializer)obj;
+ }
+
+ if ((PersistenceStrategy.BLOB == stgy || PersistenceStrategy.PRIMITIVE == stgy) && col == null)
+ col = defaultColumnName();
+ }
+
+ /**
+ * Returns java class of the object to be persisted.
+ *
+ * @return java class.
+ */
+ public Class getJavaClass() {
+ return javaCls;
+ }
+
+ /**
+ * Returns persistence strategy to use.
+ *
+ * @return persistence strategy.
+ */
+ public PersistenceStrategy getStrategy() {
+ return stgy;
+ }
+
+ /**
+ * Returns Cassandra table column name where object should be persisted in
+ * case of using BLOB or PRIMITIVE persistence strategy.
+ *
+ * @return column name.
+ */
+ public String getColumn() {
+ return col;
+ }
+
+ /**
+ * Returns serializer to be used for BLOBs.
+ *
+ * @return serializer.
+ */
+ public Serializer getSerializer() {
+ return serializer;
+ }
+
+ /**
+ * Returns a list of POJO fields to be persisted.
+ *
+ * @return list of fields.
+ */
+ public abstract List getFields();
+
+ /**
+ * Returns POJO field by Cassandra table column name.
+ *
+ * @param column column name.
+ *
+ * @return POJO field or null if not exists.
+ */
+ public PojoField getFieldByColumn(String column) {
+ List fields = getFields();
+
+ if (fields == null || fields.isEmpty())
+ return null;
+
+ for (PojoField field : fields) {
+ if (field.getColumn().equals(column))
+ return field;
+ }
+
+ return null;
+ }
+
+ /**
+ * List of POJO fields having unique mapping to Cassandra columns - skipping aliases pointing
+ * to the same Cassandra table column.
+ *
+ * @return List of fields.
+ */
+ public List cassandraUniqueFields() {
+ return casUniqueFields;
+ }
+
+ /**
+ * Returns set of database column names, used to persist field values
+ *
+ * @return set of database column names
+ */
+ public List getTableColumns() {
+ return tableColumns;
+ }
+
+ /**
+ * Returns Cassandra table columns DDL, corresponding to POJO fields which should be persisted.
+ *
+ * @return DDL statement for Cassandra table fields.
+ */
+ public String getTableColumnsDDL() {
+ return getTableColumnsDDL(null);
+ }
+
+ /**
+ * Returns Cassandra table columns DDL, corresponding to POJO fields which should be persisted.
+ *
+ * @param ignoreColumns Table columns to ignore (exclude) from DDL.
+ * @return DDL statement for Cassandra table fields.
+ */
+ public String getTableColumnsDDL(Set ignoreColumns) {
+ if (PersistenceStrategy.BLOB == stgy)
+ return " \"" + col + "\" " + DataType.Name.BLOB.toString();
+
+ if (PersistenceStrategy.PRIMITIVE == stgy)
+ return " \"" + col + "\" " + PropertyMappingHelper.getCassandraType(javaCls);
+
+ List fields = getFields();
+
+ if (fields == null || fields.isEmpty()) {
+ throw new IllegalStateException("There are no POJO fields found for '" + javaCls.toString()
+ + "' class to be presented as a Cassandra primary key");
+ }
+
+ // Accumulating already processed columns in the set, to prevent duplicating columns
+ // shared by two different POJO fields.
+ Set processedColumns = new HashSet<>();
+
+ StringBuilder builder = new StringBuilder();
+
+ for (F field : fields) {
+ if ((ignoreColumns != null && ignoreColumns.contains(field.getColumn())) ||
+ processedColumns.contains(field.getColumn())) {
+ continue;
+ }
+
+ if (builder.length() > 0)
+ builder.append(",\n");
+
+ builder.append(" ").append(field.getColumnDDL());
+
+ processedColumns.add(field.getColumn());
+ }
+
+ return builder.toString();
+ }
+
+ /**
+ * Returns default name for Cassandra column (if it's not specified explicitly).
+ *
+ * @return column name
+ */
+ protected abstract String defaultColumnName();
+
+ /**
+ * Creates instance of {@link PojoField} based on it's description in XML element.
+ *
+ * @param el XML element describing POJO field
+ * @param clazz POJO java class.
+ */
+ protected abstract F createPojoField(Element el, Class clazz);
+
+ /**
+ * Creates instance of {@link PojoField} from its field accessor.
+ *
+ * @param accessor field accessor.
+ */
+ protected abstract F createPojoField(PojoFieldAccessor accessor);
+
+ /**
+ * Creates instance of {@link PojoField} based on the other instance and java class
+ * to initialize accessor.
+ *
+ * @param field PojoField instance
+ * @param clazz java class
+ */
+ protected abstract F createPojoField(F field, Class clazz);
+
+ /**
+ * Class instance initialization.
+ */
+ protected void init() {
+ if (getColumn() != null && !getColumn().trim().isEmpty()) {
+ tableColumns = new LinkedList<>();
+ tableColumns.add(getColumn());
+ tableColumns = Collections.unmodifiableList(tableColumns);
+
+ return;
+ }
+
+ List fields = getFields();
+
+ if (fields == null || fields.isEmpty())
+ return;
+
+ tableColumns = new LinkedList<>();
+ casUniqueFields = new LinkedList<>();
+
+ for (F field : fields) {
+ if (!tableColumns.contains(field.getColumn())) {
+ tableColumns.add(field.getColumn());
+ casUniqueFields.add(field);
+ }
+ }
+
+ tableColumns = Collections.unmodifiableList(tableColumns);
+ casUniqueFields = Collections.unmodifiableList(casUniqueFields);
+ }
+
+ /**
+ * Checks if there are POJO filed with the same name or same Cassandra column specified in persistence settings.
+ *
+ * @param fields List of fields to be persisted into Cassandra.
+ */
+ protected void checkDuplicates(List fields) {
+ if (fields == null || fields.isEmpty())
+ return;
+
+ for (PojoField field1 : fields) {
+ boolean sameNames = false;
+ boolean sameCols = false;
+
+ for (PojoField field2 : fields) {
+ if (field1.getName().equals(field2.getName())) {
+ if (sameNames) {
+ throw new IllegalArgumentException("Incorrect Cassandra persistence settings, " +
+ "two POJO fields with the same name '" + field1.getName() + "' specified");
+ }
+
+ sameNames = true;
+ }
+
+ if (field1.getColumn().equals(field2.getColumn())) {
+ if (sameCols && !CassandraHelper.isCassandraCompatibleTypes(field1.getJavaClass(), field2.getJavaClass())) {
+ throw new IllegalArgumentException("Field '" + field1.getName() + "' shares the same Cassandra table " +
+ "column '" + field1.getColumn() + "' with field '" + field2.getName() + "', but their Java " +
+ "classes are different. Fields sharing the same column should have the same " +
+ "Java class as their type or should be mapped to the same Cassandra primitive type.");
+ }
+
+ sameCols = true;
+ }
+ }
+ }
+ }
+
+ /**
+ * Extracts POJO fields from a list of corresponding XML field nodes.
+ *
+ * @param fieldNodes Field nodes to process.
+ * @return POJO fields list.
+ */
+ protected List detectPojoFields(NodeList fieldNodes) {
+ List detectedFields = new LinkedList<>();
+
+ if (fieldNodes != null && fieldNodes.getLength() != 0) {
+ int cnt = fieldNodes.getLength();
+
+ for (int i = 0; i < cnt; i++) {
+ F field = createPojoField((Element)fieldNodes.item(i), getJavaClass());
+
+ // Just checking that such field exists in the class
+ PropertyMappingHelper.getPojoFieldAccessor(getJavaClass(), field.getName());
+
+ detectedFields.add(field);
+ }
+
+ return detectedFields;
+ }
+
+ PropertyDescriptor[] descriptors = PropertyUtils.getPropertyDescriptors(getJavaClass());
+
+ // Collecting Java Beans property descriptors
+ if (descriptors != null) {
+ for (PropertyDescriptor desc : descriptors) {
+ // Skip POJO field if it's read-only
+ if (desc.getWriteMethod() != null) {
+ Field field = null;
+
+ try {
+ field = getJavaClass().getDeclaredField(desc.getName());
+ }
+ catch (Throwable ignore) {
+ }
+
+ detectedFields.add(createPojoField(new PojoFieldAccessor(desc, field)));
+ }
+ }
+ }
+
+ Field[] fields = getJavaClass().getDeclaredFields();
+
+ // Collecting all fields annotated with @QuerySqlField
+ if (fields != null) {
+ for (Field field : fields) {
+ if (field.getAnnotation(QuerySqlField.class) != null && !PojoField.containsField(detectedFields, field.getName()))
+ detectedFields.add(createPojoField(new PojoFieldAccessor(field)));
+ }
+ }
+
+ return detectedFields;
+ }
+
+ /**
+ * Instantiates Class object for particular class
+ *
+ * @param clazz class name
+ * @return Class object
+ */
+ private Class getClassInstance(String clazz) {
+ try {
+ return Class.forName(clazz);
+ }
+ catch (ClassNotFoundException ignored) {
+ }
+
+ try {
+ return Class.forName(clazz, true, Thread.currentThread().getContextClassLoader());
+ }
+ catch (ClassNotFoundException ignored) {
+ }
+
+ try {
+ return Class.forName(clazz, true, PersistenceSettings.class.getClassLoader());
+ }
+ catch (ClassNotFoundException ignored) {
+ }
+
+ try {
+ return Class.forName(clazz, true, ClassLoader.getSystemClassLoader());
+ }
+ catch (ClassNotFoundException ignored) {
+ }
+
+ throw new IgniteException("Failed to load class '" + clazz + "' using reflection");
+ }
+
+ /**
+ * Creates new object instance of particular class
+ *
+ * @param clazz class name
+ * @return object
+ */
+ private Object newObjectInstance(String clazz) {
+ try {
+ return getClassInstance(clazz).newInstance();
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to instantiate class '" + clazz + "' using default constructor", e);
+ }
+ }
+
+ /**
+ * @see java.io.Serializable
+ */
+ private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+ casUniqueFields = Collections.unmodifiableList(enrichFields(casUniqueFields));
+ }
+
+ /**
+ * Sets accessor for the given {@code src} fields.
+ * Required as accessor is transient and is not present
+ * after deserialization.
+ */
+ protected List enrichFields(List src) {
+ if (src != null) {
+ List enriched = new ArrayList<>();
+
+ for (F sourceField : src)
+ enriched.add(createPojoField(sourceField, getJavaClass()));
+
+ return enriched;
+ }
+ else
+ return new ArrayList<>();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java
new file mode 100644
index 000000000..4b1e2d827
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PersistenceStrategy.java
@@ -0,0 +1,62 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+/**
+ * Describes persistence strategy to be used to persist object data into Cassandra.
+ */
+public enum PersistenceStrategy {
+ /**
+ * Stores object value as is, by mapping its value to Cassandra table column with corresponding type.
+ *
+ * Could be used for primitive java type (like Integer, String, Long and etc) which could be directly mapped
+ * to appropriate Cassandra types.
+ */
+ PRIMITIVE,
+
+ /**
+ * Stores object value as BLOB, by mapping its value to Cassandra table column with blob type.
+ * Could be used for any java type. Conversion of java object to BLOB is handled by specified serializer.
+ *
+ * Available serializer implementations:
+ *
+ *
+ * org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer - uses standard Java
+ * serialization framework.
+ *
+ *
+ * org.apache.ignite.cache.store.cassandra.serializer.KryoSerializer - uses Kryo serialization
+ * framework.
+ *
+ *
+ */
+ BLOB,
+
+ /**
+ * Stores each field of an object as a column having corresponding type in Cassandra table.
+ * Provides ability to utilize Cassandra secondary indexes for object fields.
+ *
+ * Could be used for objects which follow JavaBeans convention and having empty public constructor.
+ * Object fields should be:
+ *
+ * Primitive java types like int, long, String and etc.
+ * Collections of primitive java types like List, Map, Set
+ *
+ */
+ POJO
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java
new file mode 100644
index 000000000..facd48c21
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoField.java
@@ -0,0 +1,238 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.io.Serializable;
+import java.lang.annotation.Annotation;
+import java.util.List;
+import com.datastax.driver.core.DataType;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.store.cassandra.common.PropertyMappingHelper;
+import org.apache.ignite.cache.store.cassandra.serializer.Serializer;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for particular field in a POJO object, specifying how this field
+ * should be written to or loaded from Cassandra.
+ */
+public abstract class PojoField implements Serializable {
+ /** Name attribute of XML element describing Pojo field. */
+ private static final String NAME_ATTR = "name";
+
+ /** Column attribute of XML element describing Pojo field. */
+ private static final String COLUMN_ATTR = "column";
+
+ /** Field name. */
+ private String name;
+
+ /** Field column name in Cassandra table. */
+ private String col;
+
+ /** Field column DDL. */
+ private String colDDL;
+
+ /** Indicator for calculated field. */
+ private Boolean calculated;
+
+ /** Field property accessor. */
+ private transient PojoFieldAccessor accessor;
+
+ /**
+ * Checks if list contains POJO field with the specified name.
+ *
+ * @param fields list of POJO fields.
+ * @param fieldName field name.
+ * @return true if list contains field or false otherwise.
+ */
+ public static boolean containsField(List extends PojoField> fields, String fieldName) {
+ if (fields == null || fields.isEmpty())
+ return false;
+
+ for (PojoField field : fields) {
+ if (field.getName().equals(fieldName))
+ return true;
+ }
+
+ return false;
+ }
+
+ /**
+ * Creates instance of {@link PojoField} based on it's description in XML element.
+ *
+ * @param el XML element describing Pojo field
+ * @param pojoCls Pojo java class.
+ */
+ public PojoField(Element el, Class> pojoCls) {
+ if (el == null)
+ throw new IllegalArgumentException("DOM element representing POJO field object can't be null");
+
+ if (!el.hasAttribute(NAME_ATTR)) {
+ throw new IllegalArgumentException("DOM element representing POJO field object should have '"
+ + NAME_ATTR + "' attribute");
+ }
+
+ this.name = el.getAttribute(NAME_ATTR).trim();
+ this.col = el.hasAttribute(COLUMN_ATTR) ? el.getAttribute(COLUMN_ATTR).trim() : name.toLowerCase();
+
+ init(PropertyMappingHelper.getPojoFieldAccessor(pojoCls, name));
+ }
+
+ /**
+ * Creates instance of {@link PojoField} from its field accessor.
+ *
+ * @param accessor field accessor.
+ */
+ public PojoField(PojoFieldAccessor accessor) {
+ this.name = accessor.getName();
+
+ QuerySqlField sqlField = (QuerySqlField)accessor.getAnnotation(QuerySqlField.class);
+
+ col = sqlField != null && sqlField.name() != null && !sqlField.name().isEmpty() ?
+ sqlField.name() : name.toLowerCase();
+
+ init(accessor);
+ }
+
+ /**
+ * Creates instance of {@link PojoField} from the other instance
+ * and java class.
+ *
+ * @param field {@link PojoField} instance to copy from.
+ * @param pojoCls Class of the {@link PojoField} instance.
+ */
+ public PojoField(PojoField field, Class> pojoCls) {
+ this.name = field.name;
+ this.col = field.col;
+ this.colDDL = field.colDDL;
+
+ init(PropertyMappingHelper.getPojoFieldAccessor(pojoCls, name));
+ }
+
+ /**
+ * @return field name.
+ */
+ public String getName() {
+ return name;
+ }
+
+ /**
+ * Returns java class of the field.
+ *
+ * @return Java class.
+ */
+ public Class getJavaClass() {
+ return accessor.getFieldType();
+ }
+
+ /**
+ * @return Cassandra table column name.
+ */
+ public String getColumn() {
+ return col;
+ }
+
+ /**
+ * @return Cassandra table column DDL statement.
+ */
+ public String getColumnDDL() {
+ return colDDL;
+ }
+
+ /**
+ * Indicates if it's a calculated field - field which value just generated based on other field values.
+ * Such field will be stored in Cassandra as all other POJO fields, but it's value shouldn't be read from
+ * Cassandra - cause it's again just generated based on other field values. One of the good applications of such
+ * kind of fields - Cassandra materialized views build on top of other tables.
+ *
+ * @return {@code true} if it's auto generated field, {@code false} if not.
+ */
+ public boolean calculatedField() {
+ if (calculated != null)
+ return calculated;
+
+ return calculated = accessor.isReadOnly();
+ }
+
+ /**
+ * Gets field value as an object having Cassandra compatible type.
+ * This it could be stored directly into Cassandra without any conversions.
+ *
+ * @param obj Object instance.
+ * @param serializer {@link org.apache.ignite.cache.store.cassandra.serializer.Serializer} to use.
+ * @return Object to store in Cassandra table column.
+ */
+ public Object getValueFromObject(Object obj, Serializer serializer) {
+ Object val = accessor.getValue(obj);
+
+ if (val == null)
+ return null;
+
+ DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(val.getClass());
+
+ if (cassandraType != null)
+ return val;
+
+ if (serializer == null) {
+ throw new IllegalStateException("Can't serialize value from object '" +
+ val.getClass().getName() + "' field '" + name + "', cause there is no BLOB serializer specified");
+ }
+
+ return serializer.serialize(val);
+ }
+
+ /**
+ * Returns POJO field annotation.
+ *
+ * @param clazz Class of the annotation to get.
+ * @return annotation.
+ */
+ public Annotation getAnnotation(Class clazz) {
+ return accessor.getAnnotation(clazz);
+ }
+
+ /**
+ * Sets object field value from a {@link com.datastax.driver.core.Row} returned by Cassandra CQL statement.
+ *
+ * @param row {@link com.datastax.driver.core.Row}
+ * @param obj object which field should be populated from {@link com.datastax.driver.core.Row}
+ * @param serializer {@link org.apache.ignite.cache.store.cassandra.serializer.Serializer} to use.
+ */
+ public void setValueFromRow(Row row, Object obj, Serializer serializer) {
+ if (calculatedField())
+ return;
+
+ Object val = PropertyMappingHelper.getCassandraColumnValue(row, col, accessor.getFieldType(), serializer);
+
+ accessor.setValue(obj, val);
+ }
+
+ /**
+ * Initializes field info from property descriptor.
+ *
+ * @param accessor {@link PojoFieldAccessor} accessor.
+ */
+ private void init(PojoFieldAccessor accessor) {
+ DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(accessor.getFieldType());
+ cassandraType = cassandraType == null ? DataType.Name.BLOB : cassandraType;
+
+ this.colDDL = "\"" + col + "\" " + cassandraType.toString();
+
+ this.accessor = accessor;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoFieldAccessor.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoFieldAccessor.java
new file mode 100644
index 000000000..c8ff3e54d
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoFieldAccessor.java
@@ -0,0 +1,161 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.beans.PropertyDescriptor;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Field;
+import org.apache.ignite.IgniteException;
+
+/**
+ * Property accessor provides read/write access to POJO object properties defined through:
+ * 1) Getter/setter methods
+ * 2) Raw class members
+ */
+public class PojoFieldAccessor {
+ /** Java Bean property descriptor */
+ private PropertyDescriptor desc;
+
+ /** Object field associated with property descriptor. Used just to get annotations which
+ * applied not to property descriptor, but directly to object field associated with the property. */
+ private Field descField;
+
+ /** Object field */
+ private Field field;
+
+ /**
+ * Constructs object instance from Java Bean property descriptor, providing access to getter/setter.
+ *
+ * @param desc Java Bean property descriptor.
+ * @param field object field associated with property descriptor.
+ */
+ public PojoFieldAccessor(PropertyDescriptor desc, Field field) {
+ if (desc.getReadMethod() == null) {
+ throw new IllegalArgumentException("Field '" + desc.getName() +
+ "' of the class instance '" + desc.getPropertyType().getName() +
+ "' doesn't provide getter method");
+ }
+
+ desc.getReadMethod().setAccessible(true);
+
+ if (desc.getWriteMethod() != null)
+ desc.getWriteMethod().setAccessible(true);
+
+ this.desc = desc;
+ this.descField = field;
+ }
+
+ /**
+ * Constructs object instance from Field, providing direct access to class member.
+ *
+ * @param field Field descriptor.
+ */
+ public PojoFieldAccessor(Field field) {
+ field.setAccessible(true);
+ this.field = field;
+ }
+
+ /**
+ * Returns POJO field name.
+ *
+ * @return field name.
+ */
+ public String getName() {
+ return desc != null ? desc.getName() : field.getName();
+ }
+
+ /**
+ * Indicates if it's read-only field.
+ *
+ * @return true if field read-only, false if not.
+ */
+ public boolean isReadOnly() {
+ return desc != null && desc.getWriteMethod() == null;
+ }
+
+ /**
+ * Returns POJO field annotation.
+ *
+ * @param clazz Class of the annotation to get.
+ * @return annotation.
+ */
+ public Annotation getAnnotation(Class clazz) {
+ if (field != null)
+ return field.getAnnotation(clazz);
+
+ Annotation ann = desc.getReadMethod().getAnnotation(clazz);
+
+ if (ann != null)
+ return ann;
+
+ ann = desc.getWriteMethod() == null ? null : desc.getWriteMethod().getAnnotation(clazz);
+
+ if (ann != null)
+ return ann;
+
+ return descField == null ? null : descField.getAnnotation(clazz);
+ }
+
+ /**
+ * Returns field value for the object instance.
+ *
+ * @param obj object instance.
+ * @return field value.
+ */
+ public Object getValue(Object obj) {
+ try {
+ return desc != null ? desc.getReadMethod().invoke(obj) : field.get(obj);
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to get value of the field '" + getName() + "' from the instance " +
+ " of '" + obj.getClass().toString() + "' class", e);
+ }
+ }
+
+ /**
+ * Assigns value for the object field.
+ *
+ * @param obj object instance.
+ * @param val value to assign.
+ */
+ public void setValue(Object obj, Object val) {
+ if (isReadOnly())
+ throw new IgniteException("Can't assign value to read-only field '" + getName() + "' of the instance " +
+ " of '" + obj.getClass().toString() + "' class");
+
+ try {
+ if (desc != null)
+ desc.getWriteMethod().invoke(obj, val);
+ else
+ field.set(obj, val);
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to set value of the field '" + getName() + "' of the instance " +
+ " of '" + obj.getClass().toString() + "' class", e);
+ }
+ }
+
+ /**
+ * Returns field type.
+ *
+ * @return field type.
+ */
+ public Class getFieldType() {
+ return desc != null ? desc.getPropertyType() : field.getType();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java
new file mode 100644
index 000000000..2b02fe5c4
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoKeyField.java
@@ -0,0 +1,97 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for Ignite key POJO class
+ */
+public class PojoKeyField extends PojoField {
+ /**
+ * Specifies sort order for POJO key field
+ */
+ public enum SortOrder {
+ /** Ascending sort order. */
+ ASC,
+ /** Descending sort order. */
+ DESC
+ }
+
+ /** Xml attribute specifying sort order. */
+ private static final String SORT_ATTR = "sort";
+
+ /** Sort order. */
+ private SortOrder sortOrder;
+
+ /**
+ * Constructs Ignite cache key POJO object descriptor.
+ *
+ * @param el xml configuration element.
+ * @param pojoCls java class of key POJO field.
+ */
+ public PojoKeyField(Element el, Class pojoCls) {
+ super(el, pojoCls);
+
+ if (el.hasAttribute(SORT_ATTR)) {
+ try {
+ sortOrder = SortOrder.valueOf(el.getAttribute(SORT_ATTR).trim().toUpperCase());
+ }
+ catch (IllegalArgumentException ignored) {
+ throw new IllegalArgumentException("Incorrect sort order '" + el.getAttribute(SORT_ATTR) + "' specified");
+ }
+ }
+ }
+
+ /**
+ * Constructs instance of {@code PojoKeyField} based on the other instance and java class
+ * to initialize accessor.
+ *
+ * @param field PojoKeyField instance
+ * @param pojoCls java class of the corresponding POJO
+ */
+ public PojoKeyField(PojoKeyField field, Class> pojoCls) {
+ super(field, pojoCls);
+
+ sortOrder = field.sortOrder;
+ }
+
+ /**
+ * Constructs Ignite cache key POJO object descriptor.
+ *
+ * @param accessor property descriptor.
+ */
+ public PojoKeyField(PojoFieldAccessor accessor) {
+ super(accessor);
+
+ QuerySqlField sqlField = (QuerySqlField)accessor.getAnnotation(QuerySqlField.class);
+
+ if (sqlField != null && sqlField.descending())
+ sortOrder = SortOrder.DESC;
+ }
+
+ /**
+ * Returns sort order for the field.
+ *
+ * @return sort order.
+ */
+ public SortOrder getSortOrder() {
+ return sortOrder;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java
new file mode 100644
index 000000000..0427e6cd7
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/PojoValueField.java
@@ -0,0 +1,157 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.w3c.dom.Element;
+
+/**
+ * Descriptor for Ignite value POJO class
+ */
+public class PojoValueField extends PojoField {
+ /** Xml attribute specifying that Cassandra column is static. */
+ private static final String STATIC_ATTR = "static";
+
+ /** Xml attribute specifying that secondary index should be created for Cassandra column. */
+ private static final String INDEX_ATTR = "index";
+
+ /** Xml attribute specifying secondary index custom class. */
+ private static final String INDEX_CLASS_ATTR = "indexClass";
+
+ /** Xml attribute specifying secondary index options. */
+ private static final String INDEX_OPTIONS_ATTR = "indexOptions";
+
+ /** Indicates if Cassandra column should be indexed. */
+ private Boolean isIndexed;
+
+ /** Custom java class for Cassandra secondary index. */
+ private String idxCls;
+
+ /** Secondary index options. */
+ private String idxOptions;
+
+ /** Indicates if Cassandra column is static. */
+ private Boolean isStatic;
+
+ /**
+ * Constructs Ignite cache value field descriptor.
+ *
+ * @param el field descriptor xml configuration element.
+ * @param pojoCls field java class
+ */
+ public PojoValueField(Element el, Class pojoCls) {
+ super(el, pojoCls);
+
+ if (el.hasAttribute(STATIC_ATTR))
+ isStatic = Boolean.parseBoolean(el.getAttribute(STATIC_ATTR).trim().toLowerCase());
+
+ if (el.hasAttribute(INDEX_ATTR))
+ isIndexed = Boolean.parseBoolean(el.getAttribute(INDEX_ATTR).trim().toLowerCase());
+
+ if (el.hasAttribute(INDEX_CLASS_ATTR))
+ idxCls = el.getAttribute(INDEX_CLASS_ATTR).trim();
+
+ if (el.hasAttribute(INDEX_OPTIONS_ATTR)) {
+ idxOptions = el.getAttribute(INDEX_OPTIONS_ATTR).trim();
+
+ if (!idxOptions.toLowerCase().startsWith("with")) {
+ idxOptions = idxOptions.toLowerCase().startsWith("options") ?
+ "with " + idxOptions :
+ "with options = " + idxOptions;
+ }
+ }
+ }
+
+ /**
+ * Constructs Ignite cache value field descriptor.
+ *
+ * @param accessor field property accessor.
+ */
+ public PojoValueField(PojoFieldAccessor accessor) {
+ super(accessor);
+
+ QuerySqlField sqlField = (QuerySqlField)accessor.getAnnotation(QuerySqlField.class);
+
+ isIndexed = sqlField != null && sqlField.index();
+ }
+
+ /**
+ * Constructs instance of {@code PojoValueField} based on the other instance and java class
+ * to initialize accessor.
+ *
+ * @param field PojoValueField instance
+ * @param pojoCls java class of the corresponding POJO
+ */
+ public PojoValueField(PojoValueField field, Class> pojoCls) {
+ super(field, pojoCls);
+
+ isStatic = field.isStatic;
+ isIndexed = field.isIndexed;
+ idxCls = field.idxCls;
+ idxOptions = field.idxOptions;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getColumnDDL() {
+ String colDDL = super.getColumnDDL();
+
+ if (isStatic != null && isStatic)
+ colDDL += " static";
+
+ return colDDL;
+ }
+
+ /**
+ * Indicates if secondary index should be created for the field.
+ *
+ * @return true/false if secondary index should/shouldn't be created for the field.
+ */
+ public boolean isIndexed() {
+ return isIndexed != null && isIndexed;
+ }
+
+ /**
+ * Returns DDL for the field secondary index.
+ *
+ * @param keyspace Cassandra keyspace where index should be created.
+ * @param tbl Cassandra table for which secondary index should be created.
+ *
+ * @return secondary index DDL.
+ */
+ public String getIndexDDL(String keyspace, String tbl) {
+ if (isIndexed == null || !isIndexed)
+ return null;
+
+ StringBuilder builder = new StringBuilder();
+
+ if (idxCls != null)
+ builder.append("create custom index if not exists on \"").append(keyspace).append("\".\"").append(tbl).append("\"");
+ else
+ builder.append("create index if not exists on \"").append(keyspace).append("\".\"").append(tbl).append("\"");
+
+ builder.append(" (\"").append(getColumn()).append("\")");
+
+ if (idxCls != null)
+ builder.append(" using '").append(idxCls).append("'");
+
+ if (idxOptions != null)
+ builder.append(" ").append(idxOptions);
+
+ return builder.append(";").toString();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java
new file mode 100644
index 000000000..5e106af70
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/ValuePersistenceSettings.java
@@ -0,0 +1,100 @@
+/*
+ * 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.ignite.cache.store.cassandra.persistence;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+
+/**
+ * Stores persistence settings for Ignite cache value
+ */
+public class ValuePersistenceSettings extends PersistenceSettings {
+ /** XML element describing value field settings. */
+ private static final String FIELD_ELEMENT = "field";
+
+ /** Value fields. */
+ private List fields = new LinkedList<>();
+
+ /**
+ * Creates class instance from XML configuration.
+ *
+ * @param el XML element describing value persistence settings.
+ */
+ public ValuePersistenceSettings(Element el) {
+ super(el);
+
+ if (PersistenceStrategy.POJO != getStrategy()) {
+ init();
+
+ return;
+ }
+
+ NodeList nodes = el.getElementsByTagName(FIELD_ELEMENT);
+
+ fields = detectPojoFields(nodes);
+
+ if (fields.isEmpty())
+ throw new IllegalStateException("Failed to initialize value fields for class '" + getJavaClass().getName() + "'");
+
+ checkDuplicates(fields);
+
+ init();
+ }
+
+ /**
+ * @return List of value fields.
+ */
+ @Override public List getFields() {
+ return fields == null ? null : Collections.unmodifiableList(fields);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected String defaultColumnName() {
+ return "value";
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoValueField createPojoField(Element el, Class clazz) {
+ return new PojoValueField(el, clazz);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoValueField createPojoField(PojoFieldAccessor accessor) {
+ return new PojoValueField(accessor);
+ }
+
+ /** {@inheritDoc} */
+ @Override protected PojoValueField createPojoField(PojoValueField field, Class clazz) {
+ return new PojoValueField(field, clazz);
+ }
+
+ /**
+ * @see java.io.Serializable
+ */
+ private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException {
+ in.defaultReadObject();
+
+ fields = enrichFields(fields);
+ }
+
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java
new file mode 100644
index 000000000..7dd0840ae
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/persistence/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains persistent settings configuration
+ */
+
+package org.apache.ignite.cache.store.cassandra.persistence;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java
new file mode 100644
index 000000000..44d2d4701
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/JavaSerializer.java
@@ -0,0 +1,80 @@
+/*
+ * 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.ignite.cache.store.cassandra.serializer;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Serializer based on standard Java serialization.
+ */
+public class JavaSerializer implements Serializer {
+ /** */
+ private static final int DFLT_BUFFER_SIZE = 4096;
+
+ /** {@inheritDoc} */
+ @Override public ByteBuffer serialize(Object obj) {
+ if (obj == null)
+ return null;
+
+ ByteArrayOutputStream stream = null;
+ ObjectOutputStream out = null;
+
+ try {
+ stream = new ByteArrayOutputStream(DFLT_BUFFER_SIZE);
+
+ out = new ObjectOutputStream(stream);
+ out.writeObject(obj);
+ out.flush();
+
+ return ByteBuffer.wrap(stream.toByteArray());
+ }
+ catch (IOException e) {
+ throw new IllegalStateException("Failed to serialize object of the class '" + obj.getClass().getName() + "'", e);
+ }
+ finally {
+ U.closeQuiet(out);
+ U.closeQuiet(stream);
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object deserialize(ByteBuffer buf) {
+ ByteArrayInputStream stream = null;
+ ObjectInputStream in = null;
+
+ try {
+ stream = new ByteArrayInputStream(buf.array());
+ in = new ObjectInputStream(stream);
+
+ return in.readObject();
+ }
+ catch (Throwable e) {
+ throw new IllegalStateException("Failed to deserialize object from byte stream", e);
+ }
+ finally {
+ U.closeQuiet(in);
+ U.closeQuiet(stream);
+ }
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java
new file mode 100644
index 000000000..5b8d5422b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/Serializer.java
@@ -0,0 +1,43 @@
+/*
+ * 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.ignite.cache.store.cassandra.serializer;
+
+import java.io.Serializable;
+import java.nio.ByteBuffer;
+
+/**
+ * Interface which should be implemented by all serializers responsible
+ * for writing/loading data to/from Cassandra in binary (BLOB) format.
+ */
+public interface Serializer extends Serializable {
+ /**
+ * Serializes object into byte buffer.
+ *
+ * @param obj Object to serialize.
+ * @return Byte buffer with binary data.
+ */
+ public ByteBuffer serialize(Object obj);
+
+ /**
+ * Deserializes object from byte buffer.
+ *
+ * @param buf Byte buffer.
+ * @return Deserialized object.
+ */
+ public Object deserialize(ByteBuffer buf);
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java
new file mode 100644
index 000000000..aa1bccfa0
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/serializer/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains serializers implementation, to store BLOBs into Cassandra
+ */
+
+package org.apache.ignite.cache.store.cassandra.serializer;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java
new file mode 100644
index 000000000..5d971e877
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchExecutionAssistant.java
@@ -0,0 +1,102 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Provides information for batch operations (loadAll, deleteAll, writeAll) of Ignite cache
+ * backed by {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ *
+ * @param type of the result returned from batch operation.
+ * @param type of the value used in batch operation.
+ */
+public interface BatchExecutionAssistant {
+ /**
+ * Indicates if Cassandra tables existence is required for this batch operation.
+ *
+ * @return {@code true} true if table existence required.
+ */
+ public boolean tableExistenceRequired();
+
+ /**
+ * Cassandra table to use for an operation.
+ *
+ * @return Table name.
+ */
+ public String getTable();
+
+ /**
+ * Returns unbind CLQ statement for to be executed inside batch operation.
+ *
+ * @return Unbind CQL statement.
+ */
+ public String getStatement();
+
+ /**
+ * Binds prepared statement to current Cassandra session.
+ *
+ * @param statement Statement.
+ * @param obj Parameters for statement binding.
+ * @return Bounded statement.
+ */
+ public BoundStatement bindStatement(PreparedStatement statement, V obj);
+
+ /**
+ * Returns Ignite cache key/value persistence settings.
+ *
+ * @return persistence settings.
+ */
+ public KeyValuePersistenceSettings getPersistenceSettings();
+
+ /**
+ * Display name for the batch operation.
+ *
+ * @return Operation display name.
+ */
+ public String operationName();
+
+ /**
+ * Processes particular row inside batch operation.
+ *
+ * @param row Row to process.
+ * @param seqNum Sequential number of the row.
+ */
+ public void process(Row row, int seqNum);
+
+ /**
+ * Checks if row/object with specified sequential number is already processed.
+ *
+ * @param seqNum object sequential number
+ * @return {@code true} if object is already processed
+ */
+ public boolean alreadyProcessed(int seqNum);
+
+ /**
+ * @return number of processed objects/rows.
+ */
+ public int processedCount();
+
+ /**
+ * @return batch operation result.
+ */
+ public R processedData();
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java
new file mode 100644
index 000000000..387c98f4b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/BatchLoaderAssistant.java
@@ -0,0 +1,47 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Statement;
+
+/**
+ * Provides information for loadCache operation of {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ */
+public interface BatchLoaderAssistant {
+ /**
+ * Returns name of the batch load operation.
+ *
+ * @return operation name.
+ */
+ public String operationName();
+
+ /**
+ * Returns CQL statement to use in batch load operation.
+ *
+ * @return CQL statement for batch load operation.
+ */
+ public Statement getStatement();
+
+ /**
+ * Processes each row returned by batch load operation.
+ *
+ * @param row row selected from Cassandra table.
+ */
+ public void process(Row row);
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.java
new file mode 100644
index 000000000..facfa40a4
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSession.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.ignite.cache.store.cassandra.session;
+
+import java.io.Closeable;
+import java.util.List;
+import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
+
+/**
+ * Wrapper around Cassandra driver session, to automatically handle:
+ *
+ * Keyspace and table absence exceptions
+ * Timeout exceptions
+ * Batch operations
+ *
+ */
+public interface CassandraSession extends Closeable {
+ /**
+ * Execute single synchronous operation against Cassandra database.
+ *
+ * @param assistant execution assistance to perform the main operation logic.
+ * @param type of the result returned from operation.
+ *
+ * @return result of the operation.
+ */
+ public V execute(ExecutionAssistant assistant);
+
+ /**
+ * Executes batch asynchronous operation against Cassandra database.
+ *
+ * @param assistant execution assistance to perform the main operation logic.
+ * @param data data which should be processed in batch operation.
+ * @param type of the result returned from batch operation.
+ * @param type of the value used in batch operation.
+ *
+ * @return result of the operation.
+ */
+ public R execute(BatchExecutionAssistant assistant, Iterable extends V> data);
+
+ /**
+ * Executes batch asynchronous operation to load bunch of records
+ * specified by CQL statement from Cassandra database
+ *
+ * @param assistant execution assistance to perform the main operation logic.
+ */
+ public void execute(BatchLoaderAssistant assistant);
+
+ /**
+ * Executes all the mutations performed withing Ignite transaction against Cassandra database.
+ *
+ * @param mutations Mutations.
+ */
+ public void execute(List mutations);
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java
new file mode 100644
index 000000000..53aa42474
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/CassandraSessionImpl.java
@@ -0,0 +1,1030 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+import javax.cache.Cache;
+import com.datastax.driver.core.BatchStatement;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.exceptions.AlreadyExistsException;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import com.datastax.driver.core.querybuilder.Batch;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+import org.apache.ignite.cache.store.cassandra.common.RandomSleeper;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.session.pool.SessionPool;
+import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.internal.util.typedef.internal.LT;
+
+/**
+ * Implementation for {@link org.apache.ignite.cache.store.cassandra.session.CassandraSession}.
+ */
+public class CassandraSessionImpl implements CassandraSession {
+ /** Number of CQL query execution attempts. */
+ private static final int CQL_EXECUTION_ATTEMPTS_COUNT = 20;
+
+ /** Min timeout between CQL query execution attempts. */
+ private static final int CQL_EXECUTION_ATTEMPT_MIN_TIMEOUT = 100;
+
+ /** Max timeout between CQL query execution attempts. */
+ private static final int CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT = 500;
+
+ /** Timeout increment for CQL query execution attempts. */
+ private static final int CQL_ATTEMPTS_TIMEOUT_INCREMENT = 100;
+
+ /** Cassandra cluster builder. */
+ private volatile Cluster.Builder builder;
+
+ /**
+ * Current generation number of Cassandra session. Each time session recreated its generation will be incremented.
+ * The main idea behind session generation is to track prepared statements created with old Cassandra
+ * session (which is not valid anymore) and avoid extra refresh of Cassandra session by multiple threads.
+ **/
+ private volatile Long generation = 0L;
+
+ /** Wrapped Cassandra session. **/
+ private volatile WrappedSession wrapperSes;
+
+ /** Number of references to Cassandra driver session (for multithreaded environment). */
+ private volatile int refCnt;
+
+ /** Storage for the session prepared statements */
+ private static final Map sesStatements = new HashMap<>();
+
+ /** Number of records to immediately fetch in CQL statement execution. */
+ private Integer fetchSize;
+
+ /** Consistency level for Cassandra READ operations (select). */
+ private ConsistencyLevel readConsistency;
+
+ /** Consistency level for Cassandra WRITE operations (insert/update/delete). */
+ private ConsistencyLevel writeConsistency;
+
+ /** Expiration timeout. */
+ private long expirationTimeout;
+
+ /** Logger. */
+ private IgniteLogger log;
+
+ /** Table absence error handlers counter. */
+ private final Map tblAbsenceHandlersCnt = new ConcurrentHashMap<>();
+
+ /** Lock used to synchronize multiple threads trying to do session refresh. **/
+ private final ReentrantLock refreshLock = new ReentrantLock();
+
+ /**
+ * Creates instance of Cassandra driver session wrapper.
+ *
+ * @param builder Builder for Cassandra cluster.
+ * @param fetchSize Number of rows to immediately fetch in CQL statement execution.
+ * @param readConsistency Consistency level for Cassandra READ operations (select).
+ * @param writeConsistency Consistency level for Cassandra WRITE operations (insert/update/delete).
+ * @param expirationTimeout Expiration timout.
+ * @param log Logger.
+ */
+ public CassandraSessionImpl(Cluster.Builder builder, Integer fetchSize, ConsistencyLevel readConsistency,
+ ConsistencyLevel writeConsistency, long expirationTimeout, IgniteLogger log) {
+ this.builder = builder;
+ this.fetchSize = fetchSize;
+ this.readConsistency = readConsistency;
+ this.writeConsistency = writeConsistency;
+ this.expirationTimeout = expirationTimeout;
+ this.log = log;
+ }
+
+ /** {@inheritDoc} */
+ @Override public V execute(ExecutionAssistant assistant) {
+ int attempt = 0;
+ Throwable error = null;
+ String errorMsg = "Failed to execute Cassandra CQL statement: " + assistant.getStatement();
+
+ RandomSleeper sleeper = newSleeper();
+
+ incrementSessionRefs();
+
+ try {
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ if (attempt != 0) {
+ log.warning("Trying " + (attempt + 1) + " attempt to execute Cassandra CQL statement: " +
+ assistant.getStatement());
+ }
+
+ WrappedPreparedStatement preparedSt = null;
+ WrappedSession ses = null;
+
+ try {
+ preparedSt = prepareStatement(assistant.getTable(), assistant.getStatement(),
+ assistant.getPersistenceSettings(), assistant.tableExistenceRequired());
+
+ if (preparedSt == null)
+ return null;
+
+ Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt));
+
+ ses = session();
+
+ ResultSet res = ses.execute(statement);
+
+ Row row = res == null || !res.iterator().hasNext() ? null : res.iterator().next();
+
+ return row == null ? null : assistant.process(row);
+ }
+ catch (Throwable e) {
+ error = e;
+
+ if (CassandraHelper.isTableAbsenceError(e)) {
+ if (!assistant.tableExistenceRequired()) {
+ log.warning(errorMsg, e);
+ return null;
+ }
+
+ handleTableAbsenceError(assistant.getTable(), assistant.getPersistenceSettings());
+ }
+ else if (CassandraHelper.isHostsAvailabilityError(e))
+ handleHostsAvailabilityError(ses == null ? -1 : ses.generation, e, attempt, errorMsg);
+ else if (CassandraHelper.isPreparedStatementClusterError(e))
+ handlePreparedStatementClusterError(preparedSt == null ? -1 : preparedSt.generation, e);
+ else
+ // For an error which we don't know how to handle, we will not try next attempts and terminate.
+ throw new IgniteException(errorMsg, e);
+ }
+
+ if (!CassandraHelper.isTableAbsenceError(error))
+ sleeper.sleep();
+
+ attempt++;
+ }
+ }
+ catch (Throwable e) {
+ error = e;
+ }
+ finally {
+ decrementSessionRefs();
+ }
+
+ log.error(errorMsg, error);
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /** {@inheritDoc} */
+ @Override public R execute(BatchExecutionAssistant assistant, Iterable extends V> data) {
+ if (data == null || !data.iterator().hasNext())
+ return assistant.processedData();
+
+ int attempt = 0;
+ String errorMsg = "Failed to execute Cassandra " + assistant.operationName() + " operation";
+ Throwable error = new IgniteException(errorMsg);
+
+ RandomSleeper sleeper = newSleeper();
+
+ int dataSize = 0;
+
+ incrementSessionRefs();
+
+ try {
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ if (attempt != 0) {
+ log.warning("Trying " + (attempt + 1) + " attempt to execute Cassandra batch " +
+ assistant.operationName() + " operation to process rest " +
+ (dataSize - assistant.processedCount()) + " of " + dataSize + " elements");
+ }
+
+ //clean errors info before next communication with Cassandra
+ Throwable unknownEx = null;
+ Throwable tblAbsenceEx = null;
+ Throwable hostsAvailEx = null;
+ Throwable prepStatEx = null;
+
+ List> futResults = new LinkedList<>();
+
+ WrappedPreparedStatement preparedSt = prepareStatement(assistant.getTable(), assistant.getStatement(),
+ assistant.getPersistenceSettings(), assistant.tableExistenceRequired());
+
+ if (preparedSt == null)
+ return null;
+
+ WrappedSession ses = null;
+
+ int seqNum = 0;
+
+ for (V obj : data) {
+ if (!assistant.alreadyProcessed(seqNum)) {
+ try {
+ ses = session();
+ Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt, obj));
+ ResultSetFuture fut = ses.executeAsync(statement);
+ futResults.add(new CacheEntryImpl<>(seqNum, fut));
+ }
+ catch (Throwable e) {
+ if (CassandraHelper.isTableAbsenceError(e)) {
+ // If there are table absence error and it is not required for the operation we can return.
+ if (!assistant.tableExistenceRequired())
+ return assistant.processedData();
+
+ tblAbsenceEx = e;
+ handleTableAbsenceError(assistant.getTable(), assistant.getPersistenceSettings());
+ }
+ else if (CassandraHelper.isHostsAvailabilityError(e)) {
+ hostsAvailEx = e;
+
+ // Handle host availability only once.
+ if (hostsAvailEx == null)
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+ }
+ else if (CassandraHelper.isPreparedStatementClusterError(e)) {
+ prepStatEx = e;
+
+ handlePreparedStatementClusterError(preparedSt.generation, e);
+
+ preparedSt = prepareStatement(assistant.getTable(), assistant.getStatement(),
+ assistant.getPersistenceSettings(), assistant.tableExistenceRequired());
+
+ if (preparedSt == null)
+ return null;
+ }
+ else
+ unknownEx = e;
+ }
+ }
+
+ seqNum++;
+ }
+
+ dataSize = seqNum;
+
+ // For an error which we don't know how to handle, we will not try next attempts and terminate.
+ if (unknownEx != null)
+ throw new IgniteException(errorMsg, unknownEx);
+
+ // Remembering any of last errors.
+ if (tblAbsenceEx != null)
+ error = tblAbsenceEx;
+ else if (hostsAvailEx != null)
+ error = hostsAvailEx;
+ else if (prepStatEx != null)
+ error = prepStatEx;
+
+ // Clean errors info before next communication with Cassandra.
+ unknownEx = null;
+ tblAbsenceEx = null;
+ hostsAvailEx = null;
+ prepStatEx = null;
+
+ for (Cache.Entry futureResult : futResults) {
+ try {
+ ResultSet resSet = futureResult.getValue().getUninterruptibly();
+ Row row = resSet != null && resSet.iterator().hasNext() ? resSet.iterator().next() : null;
+
+ assistant.process(row, futureResult.getKey());
+ }
+ catch (Throwable e) {
+ if (CassandraHelper.isTableAbsenceError(e))
+ tblAbsenceEx = e;
+ else if (CassandraHelper.isHostsAvailabilityError(e))
+ hostsAvailEx = e;
+ else if (CassandraHelper.isPreparedStatementClusterError(e))
+ prepStatEx = e;
+ else
+ unknownEx = e;
+ }
+ }
+
+ // For an error which we don't know how to handle, we will not try next attempts and terminate.
+ if (unknownEx != null)
+ throw new IgniteException(errorMsg, unknownEx);
+
+ // If there are no errors occurred it means that operation successfully completed and we can return.
+ if (tblAbsenceEx == null && hostsAvailEx == null && prepStatEx == null && assistant.processedCount() == dataSize)
+ return assistant.processedData();
+
+ if (tblAbsenceEx != null) {
+ // If there are table absence error and it is not required for the operation we can return.
+ if (!assistant.tableExistenceRequired())
+ return assistant.processedData();
+
+ error = tblAbsenceEx;
+ handleTableAbsenceError(assistant.getTable(), assistant.getPersistenceSettings());
+ }
+
+ if (hostsAvailEx != null) {
+ error = hostsAvailEx;
+ handleHostsAvailabilityError(ses.generation, hostsAvailEx, attempt, errorMsg);
+ }
+
+ if (prepStatEx != null) {
+ error = prepStatEx;
+ handlePreparedStatementClusterError(preparedSt.generation, prepStatEx);
+ }
+
+ if (!CassandraHelper.isTableAbsenceError(error))
+ sleeper.sleep();
+
+ attempt++;
+ }
+ }
+ catch (Throwable e) {
+ error = e;
+ }
+ finally {
+ decrementSessionRefs();
+ }
+
+ errorMsg = "Failed to process " + (dataSize - assistant.processedCount()) +
+ " of " + dataSize + " elements, during " + assistant.operationName() +
+ " operation with Cassandra";
+
+ LT.warn(log, error, errorMsg, false, false);
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute(BatchLoaderAssistant assistant) {
+ int attempt = 0;
+ String errorMsg = "Failed to execute Cassandra " + assistant.operationName() + " operation";
+ Throwable error = new IgniteException(errorMsg);
+
+ RandomSleeper sleeper = newSleeper();
+
+ incrementSessionRefs();
+
+ try {
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ if (attempt != 0)
+ log.warning("Trying " + (attempt + 1) + " attempt to load Ignite cache");
+
+ Statement statement = tuneStatementExecutionOptions(assistant.getStatement());
+
+ WrappedSession ses = null;
+
+ try {
+ ses = session();
+
+ ResultSetFuture fut = ses.executeAsync(statement);
+ ResultSet resSet = fut.getUninterruptibly();
+
+ if (resSet == null || !resSet.iterator().hasNext())
+ return;
+
+ for (Row row : resSet)
+ assistant.process(row);
+
+ return;
+ }
+ catch (Throwable e) {
+ error = e;
+
+ if (CassandraHelper.isTableAbsenceError(e))
+ return;
+ else if (CassandraHelper.isHostsAvailabilityError(e))
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+ else
+ // For an error which we don't know how to handle, we will not try next attempts and terminate.
+ throw new IgniteException(errorMsg, e);
+ }
+
+ sleeper.sleep();
+
+ attempt++;
+ }
+ }
+ catch (Throwable e) {
+ error = e;
+ }
+ finally {
+ decrementSessionRefs();
+ }
+
+ log.error(errorMsg, error);
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void execute(List mutations) {
+ if (mutations == null || mutations.isEmpty())
+ return;
+
+ Throwable error = null;
+ String errorMsg = "Failed to apply " + mutations.size() + " mutations performed withing Ignite " +
+ "transaction into Cassandra";
+
+ int attempt = 0;
+ boolean tableExistenceRequired = false;
+ Map statements = new HashMap<>();
+ Map tableSettings = new HashMap<>();
+ RandomSleeper sleeper = newSleeper();
+
+ incrementSessionRefs();
+
+ try {
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ if (attempt != 0) {
+ log.warning("Trying " + (attempt + 1) + " attempt to apply " + mutations.size() + " mutations " +
+ "performed withing Ignite transaction into Cassandra");
+ }
+
+ WrappedPreparedStatement prepStatement = null;
+ WrappedSession ses = null;
+
+ try {
+ BatchStatement batch = new BatchStatement();
+
+ // accumulating all the mutations into one Cassandra logged batch
+ for (Mutation mutation : mutations) {
+ String key = mutation.getTable() + mutation.getClass().getName();
+ prepStatement = statements.get(key);
+
+ if (prepStatement == null) {
+ prepStatement = prepareStatement(mutation.getTable(), mutation.getStatement(),
+ mutation.getPersistenceSettings(), mutation.tableExistenceRequired());
+
+ if (prepStatement != null)
+ statements.put(key, prepStatement);
+ }
+
+ if (prepStatement != null)
+ batch.add(mutation.bindStatement(prepStatement));
+
+ if (attempt == 0) {
+ if (mutation.tableExistenceRequired()) {
+ tableExistenceRequired = true;
+
+ if (!tableSettings.containsKey(mutation.getTable()))
+ tableSettings.put(mutation.getTable(), mutation.getPersistenceSettings());
+ }
+ }
+ }
+
+ // committing logged batch into Cassandra
+ if (batch.size() > 0) {
+ ses = session();
+ ses.execute(tuneStatementExecutionOptions(batch));
+ }
+
+ return;
+ }
+ catch (Throwable e) {
+ error = e;
+
+ if (CassandraHelper.isTableAbsenceError(e)) {
+ if (tableExistenceRequired) {
+ for (Map.Entry entry : tableSettings.entrySet())
+ handleTableAbsenceError(entry.getKey(), entry.getValue());
+ }
+ else
+ return;
+ }
+ else if (CassandraHelper.isHostsAvailabilityError(e)) {
+ if (handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg))
+ statements.clear();
+ }
+ else if (CassandraHelper.isPreparedStatementClusterError(e)) {
+ handlePreparedStatementClusterError(prepStatement == null ? 0 : prepStatement.generation, e);
+ statements.clear();
+ }
+ else {
+ // For an error which we don't know how to handle, we will not try next attempts and terminate.
+ throw new IgniteException(errorMsg, e);
+ }
+ }
+
+ if (!CassandraHelper.isTableAbsenceError(error))
+ sleeper.sleep();
+
+ attempt++;
+ }
+ }
+ catch (Throwable e) {
+ error = e;
+ }
+ finally {
+ decrementSessionRefs();
+ }
+
+ log.error(errorMsg, error);
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /** {@inheritDoc} */
+ @Override public synchronized void close() throws IOException {
+ if (decrementSessionRefs() == 0 && wrapperSes != null) {
+ SessionPool.put(this, wrapperSes.ses, expirationTimeout);
+ wrapperSes = null;
+ }
+ }
+
+ /**
+ * Recreates Cassandra driver session.
+ */
+ private synchronized void refresh() {
+ //make sure that session removed from the pool
+ SessionPool.get(this);
+
+ //closing and reopening session
+ if (wrapperSes != null)
+ CassandraHelper.closeSession(wrapperSes.ses);
+
+ wrapperSes = null;
+
+ session();
+ }
+
+ /**
+ * Returns Cassandra session and its generation number.
+ *
+ * @return Wrapper object providing Cassandra session and its generation number.
+ */
+ private synchronized WrappedSession session() {
+ if (wrapperSes != null)
+ return wrapperSes;
+
+ Session ses = SessionPool.get(this);
+
+ if (ses != null) {
+ this.wrapperSes = new WrappedSession(ses, generation);
+ return this.wrapperSes;
+ }
+
+ synchronized (sesStatements) {
+ sesStatements.clear();
+ }
+
+ try {
+ ses = builder.build().connect();
+ generation++;
+ this.wrapperSes = new WrappedSession(ses, generation);
+ }
+ catch (Throwable e) {
+ throw new IgniteException("Failed to establish session with Cassandra database", e);
+ }
+
+ return this.wrapperSes;
+ }
+
+ /**
+ * Increments number of references to Cassandra driver session (required for multithreaded environment).
+ */
+ private synchronized void incrementSessionRefs() {
+ refCnt++;
+ }
+
+ /**
+ * Decrements number of references to Cassandra driver session (required for multithreaded environment).
+ */
+ private synchronized int decrementSessionRefs() {
+ if (refCnt != 0)
+ refCnt--;
+
+ return refCnt;
+ }
+
+ /**
+ * Prepares CQL statement using current Cassandra driver session.
+ *
+ * @param statement CQL statement.
+ * @param settings Persistence settings.
+ * @param tblExistenceRequired Flag indicating if table existence is required for the statement.
+ * @return Prepared statement.
+ */
+ private WrappedPreparedStatement prepareStatement(String table, String statement, KeyValuePersistenceSettings settings,
+ boolean tblExistenceRequired) {
+
+ int attempt = 0;
+ Throwable error = null;
+ String errorMsg = "Failed to prepare Cassandra CQL statement: " + statement;
+
+ RandomSleeper sleeper = newSleeper();
+
+ incrementSessionRefs();
+
+ try {
+ synchronized (sesStatements) {
+ WrappedPreparedStatement wrapper = sesStatements.get(statement);
+
+ if (wrapper != null) {
+ // Prepared statement is still actual, cause it was created with the current Cassandra session.
+ if (generation == wrapper.generation)
+ return wrapper;
+ // Prepared statement is not actual anymore, cause it was created with the previous Cassandra session.
+ else
+ sesStatements.remove(statement);
+ }
+ }
+
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ WrappedSession ses = null;
+
+ try {
+ ses = session();
+
+ WrappedPreparedStatement prepStatement = ses.prepare(statement);
+
+ synchronized (sesStatements) {
+ sesStatements.put(statement, prepStatement);
+ }
+
+ return prepStatement;
+ }
+ catch (Throwable e) {
+ if (CassandraHelper.isTableAbsenceError(e)) {
+ if (!tblExistenceRequired)
+ return null;
+
+ handleTableAbsenceError(table, settings);
+ }
+ else if (CassandraHelper.isHostsAvailabilityError(e))
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+ else
+ throw new IgniteException(errorMsg, e);
+
+ error = e;
+ }
+
+ if (!CassandraHelper.isTableAbsenceError(error))
+ sleeper.sleep();
+
+ attempt++;
+ }
+ }
+ finally {
+ decrementSessionRefs();
+ }
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /**
+ * Creates Cassandra keyspace.
+ *
+ * @param settings Persistence settings.
+ */
+ private void createKeyspace(KeyValuePersistenceSettings settings) {
+ int attempt = 0;
+ Throwable error = null;
+ String errorMsg = "Failed to create Cassandra keyspace '" + settings.getKeyspace() + "'";
+
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ WrappedSession ses = null;
+
+ try {
+ ses = session();
+
+ if (log.isInfoEnabled()) {
+ log.info("-----------------------------------------------------------------------");
+ log.info("Creating Cassandra keyspace '" + settings.getKeyspace() + "'");
+ log.info("-----------------------------------------------------------------------\n\n" +
+ settings.getKeyspaceDDLStatement() + "\n");
+ log.info("-----------------------------------------------------------------------");
+ }
+
+ ses.execute(settings.getKeyspaceDDLStatement());
+
+ if (log.isInfoEnabled())
+ log.info("Cassandra keyspace '" + settings.getKeyspace() + "' was successfully created");
+
+ return;
+ }
+ catch (AlreadyExistsException ignored) {
+ if (log.isInfoEnabled())
+ log.info("Cassandra keyspace '" + settings.getKeyspace() + "' already exist");
+
+ return;
+ }
+ catch (Throwable e) {
+ if (!CassandraHelper.isHostsAvailabilityError(e))
+ throw new IgniteException(errorMsg, e);
+
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+
+ error = e;
+ }
+
+ attempt++;
+ }
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /**
+ * Creates Cassandra table.
+ *
+ * @param settings Persistence settings.
+ */
+ private void createTable(String table, KeyValuePersistenceSettings settings) {
+ int attempt = 0;
+ Throwable error = null;
+ String tableFullName = settings.getKeyspace() + "." + table;
+ String errorMsg = "Failed to create Cassandra table '" + tableFullName + "'";
+
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ WrappedSession ses = null;
+
+ try {
+ ses = session();
+
+ if (log.isInfoEnabled()) {
+ log.info("-----------------------------------------------------------------------");
+ log.info("Creating Cassandra table '" + tableFullName + "'");
+ log.info("-----------------------------------------------------------------------\n\n" +
+ settings.getTableDDLStatement(table) + "\n");
+ log.info("-----------------------------------------------------------------------");
+ }
+
+ ses.execute(settings.getTableDDLStatement(table));
+
+ if (log.isInfoEnabled())
+ log.info("Cassandra table '" + tableFullName + "' was successfully created");
+
+ return;
+ }
+ catch (AlreadyExistsException ignored) {
+ if (log.isInfoEnabled())
+ log.info("Cassandra table '" + tableFullName + "' already exist");
+
+ return;
+ }
+ catch (Throwable e) {
+ if (!CassandraHelper.isHostsAvailabilityError(e) && !CassandraHelper.isKeyspaceAbsenceError(e))
+ throw new IgniteException(errorMsg, e);
+
+ if (CassandraHelper.isKeyspaceAbsenceError(e)) {
+ log.warning("Failed to create Cassandra table '" + tableFullName +
+ "' cause appropriate keyspace doesn't exist", e);
+ createKeyspace(settings);
+ }
+ else if (CassandraHelper.isHostsAvailabilityError(e))
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+
+ error = e;
+ }
+
+ attempt++;
+ }
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /**
+ * Creates Cassandra table indexes.
+ *
+ * @param settings Persistence settings.
+ */
+ private void createTableIndexes(String table, KeyValuePersistenceSettings settings) {
+ List indexDDLStatements = settings.getIndexDDLStatements(table);
+
+ if (indexDDLStatements == null || indexDDLStatements.isEmpty())
+ return;
+
+ int attempt = 0;
+ Throwable error = null;
+ String tableFullName = settings.getKeyspace() + "." + table;
+ String errorMsg = "Failed to create indexes for Cassandra table " + tableFullName;
+
+ while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) {
+ WrappedSession ses = null;
+
+ try {
+ ses = session();
+
+ if (log.isInfoEnabled()) {
+ log.info("-----------------------------------------------------------------------");
+ log.info("Creating indexes for Cassandra table '" + tableFullName + "'");
+ log.info("-----------------------------------------------------------------------");
+ }
+
+ for (String statement : indexDDLStatements) {
+ try {
+ if (log.isInfoEnabled()) {
+ log.info(statement);
+ log.info("-----------------------------------------------------------------------");
+ }
+
+ ses.execute(statement);
+ }
+ catch (AlreadyExistsException ignored) {
+ }
+ catch (Throwable e) {
+ if (!(e instanceof InvalidQueryException) || !"Index already exists".equals(e.getMessage()))
+ throw new IgniteException(errorMsg, e);
+ }
+ }
+
+ if (log.isInfoEnabled())
+ log.info("Indexes for Cassandra table '" + tableFullName + "' were successfully created");
+
+ return;
+ }
+ catch (Throwable e) {
+ if (CassandraHelper.isHostsAvailabilityError(e))
+ handleHostsAvailabilityError(ses == null ? 0 : ses.generation, e, attempt, errorMsg);
+ else if (CassandraHelper.isTableAbsenceError(e))
+ createTable(table, settings);
+ else
+ throw new IgniteException(errorMsg, e);
+
+ error = e;
+ }
+
+ attempt++;
+ }
+
+ throw new IgniteException(errorMsg, error);
+ }
+
+ /**
+ * Tunes CQL statement execution options (consistency level, fetch option and etc.).
+ *
+ * @param statement Statement.
+ * @return Modified statement.
+ */
+ private Statement tuneStatementExecutionOptions(Statement statement) {
+ String qry = "";
+
+ if (statement instanceof BoundStatement)
+ qry = ((BoundStatement)statement).preparedStatement().getQueryString().trim().toLowerCase();
+ else if (statement instanceof PreparedStatement)
+ qry = ((PreparedStatement)statement).getQueryString().trim().toLowerCase();
+
+ boolean readStatement = qry.startsWith("select");
+ boolean writeStatement = statement instanceof Batch || statement instanceof BatchStatement ||
+ qry.startsWith("insert") || qry.startsWith("delete") || qry.startsWith("update");
+
+ if (readStatement && readConsistency != null)
+ statement.setConsistencyLevel(readConsistency);
+
+ if (writeStatement && writeConsistency != null)
+ statement.setConsistencyLevel(writeConsistency);
+
+ if (fetchSize != null)
+ statement.setFetchSize(fetchSize);
+
+ return statement;
+ }
+
+ /**
+ * Handles situation when Cassandra table doesn't exist.
+ *
+ * @param settings Persistence settings.
+ */
+ private void handleTableAbsenceError(String table, KeyValuePersistenceSettings settings) {
+ String tableFullName = settings.getKeyspace() + "." + table;
+
+ AtomicInteger counter = tblAbsenceHandlersCnt.computeIfAbsent(tableFullName, k -> new AtomicInteger(-1));
+
+ int hndNum = counter.incrementAndGet();
+
+ try {
+ synchronized (counter) {
+ // Oooops... I am not the first thread who tried to handle table absence problem.
+ if (hndNum != 0) {
+ log.warning("Table " + tableFullName + " absence problem detected. " +
+ "Another thread already fixed it.");
+ return;
+ }
+
+ log.warning("Table " + tableFullName + " absence problem detected. " +
+ "Trying to create table.");
+
+ createKeyspace(settings);
+ createTable(table, settings);
+ createTableIndexes(table, settings);
+ }
+ }
+ finally {
+ if (hndNum == 0)
+ counter.set(-1);
+ }
+ }
+
+ /**
+ * Handles situation when prepared statement execution failed cause session to the cluster was released.
+ *
+ * @param sesGeneration Generation of Cassandra session used to create prepared statement.
+ * @param e Exception thrown during statement execution.
+ */
+ private void handlePreparedStatementClusterError(long sesGeneration, Throwable e) {
+ if (sesGeneration < generation) {
+ log.warning("Prepared statement cluster error detected, another thread already fixed the problem", e);
+ return;
+ }
+
+ refreshLock.lock();
+
+ try {
+ if (sesGeneration < generation) {
+ log.warning("Prepared statement cluster error detected, another thread already fixed the problem", e);
+ return;
+ }
+
+ log.warning("Prepared statement cluster error detected, refreshing Cassandra session", e);
+
+ refresh();
+
+ log.warning("Cassandra session refreshed");
+ }
+ finally {
+ refreshLock.unlock();
+ }
+ }
+
+ /**
+ * Handles situation when Cassandra host which is responsible for CQL query execution became unavailable.
+ *
+ * @param sesGeneration Generation of Cassandra session used to run CQL statement.
+ * @param e Exception to handle.
+ * @param attempt Number of attempts.
+ * @param msg Error message.
+ * @return {@code true} if host unavailability was successfully handled.
+ */
+ private boolean handleHostsAvailabilityError(long sesGeneration, Throwable e, int attempt, String msg) {
+ if (attempt >= CQL_EXECUTION_ATTEMPTS_COUNT) {
+ log.error("Host availability problem detected. " +
+ "Number of CQL execution attempts reached maximum " + CQL_EXECUTION_ATTEMPTS_COUNT +
+ ", exception will be thrown to upper execution layer.", e);
+ throw msg == null ? new IgniteException(e) : new IgniteException(msg, e);
+ }
+
+ if (attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 4 ||
+ attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 2 ||
+ attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 2 + CQL_EXECUTION_ATTEMPTS_COUNT / 4 ||
+ attempt == CQL_EXECUTION_ATTEMPTS_COUNT - 1) {
+
+ refreshLock.lock();
+
+ try {
+ if (sesGeneration < generation)
+ log.warning("Host availability problem detected, but already handled by another thread");
+ else {
+ log.warning("Host availability problem detected, CQL execution attempt " + (attempt + 1) + ", " +
+ "refreshing Cassandra session", e);
+
+ refresh();
+
+ log.warning("Cassandra session refreshed");
+
+ return true;
+ }
+ }
+ finally {
+ refreshLock.unlock();
+ }
+ }
+
+ log.warning("Host availability problem detected, CQL execution attempt " + (attempt + 1) + ", " +
+ "sleeping extra " + CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT + " milliseconds", e);
+
+ try {
+ Thread.sleep(CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT);
+ }
+ catch (InterruptedException ignored) {
+ }
+
+ log.warning("Sleep completed");
+
+ return false;
+ }
+
+ /**
+ * @return New random sleeper.
+ */
+ private RandomSleeper newSleeper() {
+ return new RandomSleeper(CQL_EXECUTION_ATTEMPT_MIN_TIMEOUT,
+ CQL_EXECUTION_ATTEMPT_MAX_TIMEOUT,
+ CQL_ATTEMPTS_TIMEOUT_INCREMENT, log);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java
new file mode 100644
index 000000000..b0dba8bf7
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/ExecutionAssistant.java
@@ -0,0 +1,84 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.Row;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Provides information for single operations (load, delete, write) of Ignite cache
+ * backed by {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}.
+ *
+ * @param type of the result returned from operation.
+ */
+public interface ExecutionAssistant {
+ /**
+ * Indicates if Cassandra table existence is required for an operation.
+ *
+ * @return true if table existence required.
+ */
+ public boolean tableExistenceRequired();
+
+ /**
+ * Cassandra table to use for an operation.
+ *
+ * @return Table name.
+ */
+ public String getTable();
+
+ /**
+ * Returns CQL statement to be used for an operation.
+ *
+ * @return CQL statement.
+ */
+ public String getStatement();
+
+ /**
+ * Binds prepared statement.
+ *
+ * @param statement prepared statement.
+ *
+ * @return bound statement.
+ */
+ public BoundStatement bindStatement(PreparedStatement statement);
+
+ /**
+ * Persistence settings to use for an operation.
+ *
+ * @return persistence settings.
+ */
+ public KeyValuePersistenceSettings getPersistenceSettings();
+
+ /**
+ * Returns operation name.
+ *
+ * @return operation name.
+ */
+ public String operationName();
+
+ /**
+ * Processes Cassandra database table row returned by specified CQL statement.
+ *
+ * @param row Cassandra database table row.
+ *
+ * @return result of the operation.
+ */
+ public R process(Row row);
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
new file mode 100644
index 000000000..1133caf3a
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/GenericBatchExecutionAssistant.java
@@ -0,0 +1,71 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import java.util.HashSet;
+import java.util.Set;
+import com.datastax.driver.core.Row;
+
+/**
+ * Implementation of the {@link org.apache.ignite.cache.store.cassandra.session.BatchExecutionAssistant}.
+ *
+ * @param Type of the result returned from batch operation
+ * @param Type of the value used in batch operation
+ */
+public abstract class GenericBatchExecutionAssistant implements BatchExecutionAssistant {
+ /** Identifiers of already processed objects. */
+ private Set processed = new HashSet<>();
+
+ /** {@inheritDoc} */
+ @Override public void process(Row row, int seqNum) {
+ if (processed.contains(seqNum))
+ return;
+
+ process(row);
+
+ processed.add(seqNum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean alreadyProcessed(int seqNum) {
+ return processed.contains(seqNum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int processedCount() {
+ return processed.size();
+ }
+
+ /** {@inheritDoc} */
+ @Override public R processedData() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean tableExistenceRequired() {
+ return false;
+ }
+
+ /**
+ * Processes particular row inside batch operation.
+ *
+ * @param row Row to process.
+ */
+ protected void process(Row row) {
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java
new file mode 100644
index 000000000..ab0795bf4
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/LoadCacheCustomQueryWorker.java
@@ -0,0 +1,121 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import java.util.concurrent.Callable;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.Statement;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+import org.apache.ignite.lang.IgniteBiInClosure;
+
+/**
+ * Worker for load cache using custom user query.
+ *
+ * @param Key type.
+ * @param Value type.
+ */
+public class LoadCacheCustomQueryWorker implements Callable {
+ /** Cassandra session to execute CQL query */
+ private final CassandraSession ses;
+
+ /** Statement. */
+ private final Statement stmt;
+
+ /** Persistence controller */
+ private final PersistenceController ctrl;
+
+ /** Logger */
+ private final IgniteLogger log;
+
+ /** Closure for loaded values. */
+ private final IgniteBiInClosure clo;
+
+ /**
+ * @param ses Session.
+ * @param qry Query.
+ * @param ctrl Control.
+ * @param log Logger.
+ * @param clo Closure for loaded values.
+ */
+ public LoadCacheCustomQueryWorker(CassandraSession ses, String qry, PersistenceController ctrl,
+ IgniteLogger log, IgniteBiInClosure clo) {
+ this(ses, new SimpleStatement(qry.trim().endsWith(";") ? qry : qry + ';'), ctrl, log, clo);
+ }
+
+ /**
+ * @param ses Session.
+ * @param stmt Statement.
+ * @param ctrl Control.
+ * @param log Logger.
+ * @param clo Closure for loaded values.
+ */
+ public LoadCacheCustomQueryWorker(CassandraSession ses, Statement stmt, PersistenceController ctrl,
+ IgniteLogger log, IgniteBiInClosure clo) {
+ this.ses = ses;
+ this.stmt = stmt;
+ this.ctrl = ctrl;
+ this.log = log;
+ this.clo = clo;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Void call() throws Exception {
+ ses.execute(new BatchLoaderAssistant() {
+ /** {@inheritDoc} */
+ @Override public String operationName() {
+ return "loadCache";
+ }
+
+ /** {@inheritDoc} */
+ @Override public Statement getStatement() {
+ return stmt;
+ }
+
+ /** {@inheritDoc} */
+ @Override public void process(Row row) {
+ K key;
+ V val;
+
+ try {
+ key = (K)ctrl.buildKeyObject(row);
+ }
+ catch (Throwable e) {
+ log.error("Failed to build Ignite key object from provided Cassandra row", e);
+
+ throw new IgniteException("Failed to build Ignite key object from provided Cassandra row", e);
+ }
+
+ try {
+ val = (V)ctrl.buildValueObject(row);
+ }
+ catch (Throwable e) {
+ log.error("Failed to build Ignite value object from provided Cassandra row", e);
+
+ throw new IgniteException("Failed to build Ignite value object from provided Cassandra row", e);
+ }
+
+ clo.apply(key, val);
+ }
+ });
+
+ return null;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedPreparedStatement.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedPreparedStatement.java
new file mode 100644
index 000000000..46d530631
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedPreparedStatement.java
@@ -0,0 +1,180 @@
+/*
+ * 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.ignite.cache.store.cassandra.session;
+
+import java.nio.ByteBuffer;
+import java.util.Map;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.CodecRegistry;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.PreparedId;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.policies.RetryPolicy;
+
+/**
+ * Simple wrapper providing access to Cassandra prepared statement and generation of Cassandra
+ * session which was used to create this statement
+ */
+public class WrappedPreparedStatement implements PreparedStatement {
+ /** Prepared statement. **/
+ private final PreparedStatement st;
+
+ /** Generation of Cassandra session which was used to prepare this statement. **/
+ final long generation;
+
+ /**
+ * Constructor.
+ *
+ * @param st Prepared statement.
+ * @param generation Generation of Cassandra session used to prepare this statement.
+ */
+ WrappedPreparedStatement(PreparedStatement st, long generation) {
+ this.st = st;
+ this.generation = generation;
+ }
+
+ /**
+ * Getter for wrapped statement.
+ *
+ * @return Wrapped original statement.
+ */
+ public PreparedStatement getWrappedStatement() {
+ return st;
+ }
+
+ /** {@inheritDoc} */
+ @Override public ColumnDefinitions getVariables() {
+ return st.getVariables();
+ }
+
+ /** {@inheritDoc} */
+ @Override public BoundStatement bind(Object... values) {
+ return st.bind(values);
+ }
+
+ /** {@inheritDoc} */
+ @Override public BoundStatement bind() {
+ return st.bind();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setRoutingKey(ByteBuffer routingKey) {
+ return st.setRoutingKey(routingKey);
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setRoutingKey(ByteBuffer... routingKeyComponents) {
+ return st.setRoutingKey(routingKeyComponents);
+ }
+
+ /** {@inheritDoc} */
+ @Override public ByteBuffer getRoutingKey() {
+ return st.getRoutingKey();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setConsistencyLevel(ConsistencyLevel consistency) {
+ return st.setConsistencyLevel(consistency);
+ }
+
+ /** {@inheritDoc} */
+ @Override public ConsistencyLevel getConsistencyLevel() {
+ return st.getConsistencyLevel();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setSerialConsistencyLevel(ConsistencyLevel serialConsistency) {
+ return st.setSerialConsistencyLevel(serialConsistency);
+ }
+
+ /** {@inheritDoc} */
+ @Override public ConsistencyLevel getSerialConsistencyLevel() {
+ return st.getSerialConsistencyLevel();
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getQueryString() {
+ return st.getQueryString();
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getQueryKeyspace() {
+ return st.getQueryKeyspace();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement enableTracing() {
+ return st.enableTracing();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement disableTracing() {
+ return st.disableTracing();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean isTracing() {
+ return st.isTracing();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setRetryPolicy(RetryPolicy policy) {
+ return st.setRetryPolicy(policy);
+ }
+
+ /** {@inheritDoc} */
+ @Override public RetryPolicy getRetryPolicy() {
+ return st.getRetryPolicy();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedId getPreparedId() {
+ return st.getPreparedId();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Map getIncomingPayload() {
+ return st.getIncomingPayload();
+ }
+
+ /** {@inheritDoc} */
+ @Override public Map getOutgoingPayload() {
+ return st.getOutgoingPayload();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setOutgoingPayload(Map payload) {
+ return st.setOutgoingPayload(payload);
+ }
+
+ /** {@inheritDoc} */
+ @Override public CodecRegistry getCodecRegistry() {
+ return st.getCodecRegistry();
+ }
+
+ /** {@inheritDoc} */
+ @Override public PreparedStatement setIdempotent(Boolean idempotent) {
+ return st.setIdempotent(idempotent);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Boolean isIdempotent() {
+ return st.isIdempotent();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedSession.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedSession.java
new file mode 100644
index 000000000..d9b722435
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/WrappedSession.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.ignite.cache.store.cassandra.session;
+
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+
+/**
+ * Simple container for Cassandra session and its generation number.
+ */
+public class WrappedSession {
+ /** Cassandra driver session. **/
+ final Session ses;
+
+ /** Cassandra session generation number. **/
+ final long generation;
+
+ /**
+ * Constructor.
+ *
+ * @param ses Cassandra session.
+ * @param generation Cassandra session generation number.
+ */
+ WrappedSession(Session ses, long generation) {
+ this.ses = ses;
+ this.generation = generation;
+ }
+
+ /**
+ * Prepares the provided query string.
+ *
+ * @param query the CQL query string to prepare
+ * @return the prepared statement corresponding to {@code query}.
+ * @throws NoHostAvailableException if no host in the cluster can be
+ * contacted successfully to prepare this query.
+ */
+ WrappedPreparedStatement prepare(String query) {
+ return new WrappedPreparedStatement(ses.prepare(query), generation);
+ }
+
+ /**
+ * Executes the provided query.
+ *
+ * @param statement The CQL query to execute (that can be any {@link Statement}).
+ *
+ * @return The result of the query. That result will never be null but can
+ */
+ ResultSet execute(Statement statement) {
+ return ses.execute(statement);
+ }
+
+ /**
+ * Executes the provided query.
+ *
+ * @param query The CQL query to execute (that can be any {@link Statement}).
+ *
+ * @return The result of the query. That result will never be null but can
+ */
+ ResultSet execute(String query) {
+ return ses.execute(query);
+ }
+
+ /**
+ * Executes the provided query asynchronously.
+ *
+ * @param statement the CQL query to execute (that can be any {@code Statement}).
+ *
+ * @return a future on the result of the query.
+ */
+ ResultSetFuture executeAsync(Statement statement) {
+ return ses.executeAsync(statement);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/package-info.java
new file mode 100644
index 000000000..9c8b917ba
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains classes responsible for handling sessions and communication with Cassandra
+ */
+
+package org.apache.ignite.cache.store.cassandra.session;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/IdleSession.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/IdleSession.java
new file mode 100644
index 000000000..0faf4d3a9
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/IdleSession.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.ignite.cache.store.cassandra.session.pool;
+
+import com.datastax.driver.core.Session;
+import org.apache.ignite.cache.store.cassandra.common.CassandraHelper;
+
+/**
+ * Simple wrapper for idle Cassandra session returned to pool, responsible for monitoring session expiration and its closing.
+ */
+public class IdleSession {
+ /** Cassandra driver session. */
+ private Session ses;
+
+ /** Expiration timeout. */
+ private long expirationTimeout;
+
+ /** Wrapper creation time. */
+ private long time;
+
+ /**
+ * Creates instance of Cassandra driver session wrapper.
+ *
+ * @param ses Cassandra driver session.
+ * @param expirationTimeout Session expiration timeout.
+ */
+ public IdleSession(Session ses, long expirationTimeout) {
+ this.ses = ses;
+ this.expirationTimeout = expirationTimeout;
+ this.time = System.currentTimeMillis();
+ }
+
+ /**
+ * Checks if Cassandra driver session expired.
+ *
+ * @return true if session expired.
+ */
+ public boolean expired() {
+ return expirationTimeout > 0 && System.currentTimeMillis() - time > expirationTimeout;
+ }
+
+ /**
+ * Returns wrapped Cassandra driver session.
+ *
+ * @return Cassandra driver session.
+ */
+ public Session driverSession() {
+ return ses;
+ }
+
+ /**
+ * Closes wrapped Cassandra driver session
+ */
+ public void release() {
+ CassandraHelper.closeSession(ses);
+ ses = null;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java
new file mode 100644
index 000000000..3fd480170
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/SessionPool.java
@@ -0,0 +1,174 @@
+/*
+ * 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.ignite.cache.store.cassandra.session.pool;
+
+import java.lang.Thread.State;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import com.datastax.driver.core.Session;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
+
+/**
+ * Cassandra driver sessions pool.
+ */
+public class SessionPool {
+ /**
+ * Monitors session pool and closes unused session.
+ */
+ private static class SessionMonitor extends Thread {
+ /** {@inheritDoc} */
+ @Override public void run() {
+ try {
+ while (true) {
+ try {
+ Thread.sleep(SLEEP_TIMEOUT);
+ }
+ catch (InterruptedException ignored) {
+ return;
+ }
+
+ List> expiredSessions = new LinkedList<>();
+
+ int sessionsCnt;
+
+ synchronized (sessions) {
+ sessionsCnt = sessions.size();
+
+ for (Map.Entry entry : sessions.entrySet()) {
+ if (entry.getValue().expired())
+ expiredSessions.add(entry);
+ }
+
+ for (Map.Entry entry : expiredSessions)
+ sessions.remove(entry.getKey());
+ }
+
+ for (Map.Entry entry : expiredSessions)
+ entry.getValue().release();
+
+ // all sessions in the pool expired, thus we don't need additional thread to manage sessions in the pool
+ if (sessionsCnt == expiredSessions.size())
+ return;
+ }
+ }
+ finally {
+ release();
+ }
+ }
+ }
+
+ /** Sessions monitor sleep timeout. */
+ private static final long SLEEP_TIMEOUT = 60000; // 1 minute.
+
+ /** Sessions which were returned to pool. */
+ private static final Map sessions = new HashMap<>();
+
+ /** Singleton instance. */
+ private static SessionMonitor monitorSingleton;
+
+ static {
+ Runtime.getRuntime().addShutdownHook(new Thread() {
+ @Override public void run() {
+ release();
+ }
+ });
+ }
+
+ /**
+ * Returns Cassandra driver session to sessions pool.
+ *
+ * @param cassandraSes Session wrapper.
+ * @param driverSes Driver session.
+ * @param expirationTimeout Expiration timeout.
+ */
+ public static void put(CassandraSessionImpl cassandraSes, Session driverSes, long expirationTimeout) {
+ if (cassandraSes == null || driverSes == null)
+ return;
+
+ IdleSession old;
+
+ synchronized (sessions) {
+ old = sessions.put(cassandraSes, new IdleSession(driverSes, expirationTimeout));
+
+ if (monitorSingleton == null || State.TERMINATED.equals(monitorSingleton.getState())) {
+ monitorSingleton = new SessionMonitor();
+ monitorSingleton.setDaemon(true);
+ monitorSingleton.setName("Cassandra-sessions-pool");
+ monitorSingleton.start();
+ }
+ }
+
+ if (old != null)
+ old.release();
+ }
+
+ /**
+ * Extracts Cassandra driver session from pool.
+ *
+ * @param cassandraSes Session wrapper.
+ * @return Cassandra driver session.
+ */
+ public static Session get(CassandraSessionImpl cassandraSes) {
+ if (cassandraSes == null)
+ return null;
+
+ IdleSession wrapper;
+
+ synchronized (sessions) {
+ wrapper = sessions.remove(cassandraSes);
+ }
+
+ return wrapper == null ? null : wrapper.driverSession();
+ }
+
+ /**
+ * Releases all session from pool and closes all their connections to Cassandra database.
+ */
+ public static void release() {
+ Collection wrappers;
+
+ synchronized (sessions) {
+ try {
+ if (sessions.isEmpty())
+ return;
+
+ wrappers = new LinkedList<>();
+
+ for (IdleSession wrapper : sessions.values())
+ wrappers.add(wrapper);
+
+ sessions.clear();
+ }
+ finally {
+ if (!(Thread.currentThread() instanceof SessionMonitor) && monitorSingleton != null) {
+ try {
+ monitorSingleton.interrupt();
+ }
+ catch (Throwable ignored) {
+ }
+ }
+ }
+ }
+
+ for (IdleSession wrapper : wrappers)
+ wrapper.release();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/package-info.java
new file mode 100644
index 000000000..446079355
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/pool/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains session pool implenetation for Cassandra sessions
+ */
+
+package org.apache.ignite.cache.store.cassandra.session.pool;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/BaseMutation.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/BaseMutation.java
new file mode 100644
index 000000000..2625e8761
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/BaseMutation.java
@@ -0,0 +1,68 @@
+/*
+ * 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.ignite.cache.store.cassandra.session.transaction;
+
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+
+/**
+ * Base class to inherit from to implement specific mutations operation.
+ */
+public abstract class BaseMutation implements Mutation {
+ /** Cassandra table to use. */
+ private final String table;
+
+ /** Persistence controller to be utilized for mutation. */
+ private final PersistenceController ctrl;
+
+ /**
+ * Creates instance of mutation operation.
+ *
+ * @param table Cassandra table which should be used for the mutation.
+ * @param ctrl Persistence controller to use.
+ */
+ public BaseMutation(String table, PersistenceController ctrl) {
+ if (table == null || table.trim().isEmpty())
+ throw new IllegalArgumentException("Table name should be specified");
+
+ if (ctrl == null)
+ throw new IllegalArgumentException("Persistence controller should be specified");
+
+ this.table = table;
+ this.ctrl = ctrl;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getTable() {
+ return table;
+ }
+
+ /** {@inheritDoc} */
+ @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+ return ctrl.getPersistenceSettings();
+ }
+
+ /**
+ * Service method to get persistence controller instance
+ *
+ * @return Persistence controller to use for the mutation
+ */
+ protected PersistenceController controller() {
+ return ctrl;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/DeleteMutation.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/DeleteMutation.java
new file mode 100644
index 000000000..79c0bfe08
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/DeleteMutation.java
@@ -0,0 +1,57 @@
+/*
+ * 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.ignite.cache.store.cassandra.session.transaction;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+
+/**
+ * Mutation which deletes object from Cassandra.
+ */
+public class DeleteMutation extends BaseMutation {
+ /** Ignite cache key of the object which should be deleted. */
+ private final Object key;
+
+ /**
+ * Creates instance of delete mutation operation.
+ *
+ * @param key Ignite cache key of the object which should be deleted.
+ * @param table Cassandra table which should be used for the mutation.
+ * @param ctrl Persistence controller to use.
+ */
+ public DeleteMutation(Object key, String table, PersistenceController ctrl) {
+ super(table, ctrl);
+ this.key = key;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean tableExistenceRequired() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getStatement() {
+ return controller().getDeleteStatement(getTable());
+ }
+
+ /** {@inheritDoc} */
+ @Override public BoundStatement bindStatement(PreparedStatement statement) {
+ return controller().bindKey(statement, key);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/Mutation.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/Mutation.java
new file mode 100644
index 000000000..f3fb35480
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/Mutation.java
@@ -0,0 +1,63 @@
+/*
+ * 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.ignite.cache.store.cassandra.session.transaction;
+
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Provides information about particular mutation operation performed withing transaction.
+ */
+public interface Mutation {
+ /**
+ * Cassandra table to use for an operation.
+ *
+ * @return Table name.
+ */
+ public String getTable();
+
+ /**
+ * Indicates if Cassandra tables existence is required for this operation.
+ *
+ * @return {@code true} true if table existence required.
+ */
+ public boolean tableExistenceRequired();
+
+ /**
+ * Returns Ignite cache key/value persistence settings.
+ *
+ * @return persistence settings.
+ */
+ public KeyValuePersistenceSettings getPersistenceSettings();
+
+ /**
+ * Returns unbind CLQ statement for to be executed.
+ *
+ * @return Unbind CQL statement.
+ */
+ public String getStatement();
+
+ /**
+ * Binds prepared statement to current Cassandra session.
+ *
+ * @param statement Statement.
+ * @return Bounded statement.
+ */
+ public BoundStatement bindStatement(PreparedStatement statement);
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/WriteMutation.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/WriteMutation.java
new file mode 100644
index 000000000..22ecf2a78
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/WriteMutation.java
@@ -0,0 +1,58 @@
+/*
+ * 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.ignite.cache.store.cassandra.session.transaction;
+
+import javax.cache.Cache;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.PreparedStatement;
+import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
+
+/**
+ * Mutation which writes(inserts) object into Cassandra.
+ */
+public class WriteMutation extends BaseMutation {
+ /** Ignite cache entry to be inserted into Cassandra. */
+ private final Cache.Entry entry;
+
+ /**
+ * Creates instance of delete mutation operation.
+ *
+ * @param entry Ignite cache entry to be inserted into Cassandra.
+ * @param table Cassandra table which should be used for the mutation.
+ * @param ctrl Persistence controller to use.
+ */
+ public WriteMutation(Cache.Entry entry, String table, PersistenceController ctrl) {
+ super(table, ctrl);
+ this.entry = entry;
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean tableExistenceRequired() {
+ return true;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getStatement() {
+ return controller().getWriteStatement(getTable());
+ }
+
+ /** {@inheritDoc} */
+ @Override public BoundStatement bindStatement(PreparedStatement statement) {
+ return controller().bindKeyValue(statement, entry.getKey(), entry.getValue());
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/package-info.java
new file mode 100644
index 000000000..e4d437716
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/session/transaction/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains mutations implementation, to store changes made inside Ignite transaction
+ */
+
+package org.apache.ignite.cache.store.cassandra.session.transaction;
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java
new file mode 100644
index 000000000..569c65de5
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/DDLGenerator.java
@@ -0,0 +1,83 @@
+/*
+ * 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.ignite.cache.store.cassandra.utils;
+
+import java.io.File;
+import java.util.List;
+
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+
+/**
+ * Generates Cassandra DDL statements from persistence descriptor xml file.
+ */
+public class DDLGenerator {
+ /**
+ * DDLGenerator entry point.
+ *
+ * @param args Arguments for DDLGenerator.
+ */
+ public static void main(String[] args) {
+ if (args == null || args.length == 0)
+ return;
+
+ boolean success = true;
+
+ for (String arg : args) {
+ File file = new File(arg);
+ if (!file.isFile()) {
+ success = false;
+ System.out.println("-------------------------------------------------------------");
+ System.out.println("Incorrect file specified: " + arg);
+ System.out.println("-------------------------------------------------------------");
+ continue;
+ }
+
+ try {
+ KeyValuePersistenceSettings settings = new KeyValuePersistenceSettings(file);
+ String table = settings.getTable() != null ? settings.getTable() : "my_table";
+
+ System.out.println("-------------------------------------------------------------");
+ System.out.println("DDL for keyspace/table from file: " + arg);
+ System.out.println("-------------------------------------------------------------");
+ System.out.println();
+ System.out.println(settings.getKeyspaceDDLStatement());
+ System.out.println();
+ System.out.println(settings.getTableDDLStatement(table));
+ System.out.println();
+
+ List statements = settings.getIndexDDLStatements(table);
+ if (statements != null && !statements.isEmpty()) {
+ for (String st : statements) {
+ System.out.println(st);
+ System.out.println();
+ }
+ }
+ }
+ catch (Throwable e) {
+ success = false;
+ System.out.println("-------------------------------------------------------------");
+ System.out.println("Invalid file specified: " + arg);
+ System.out.println("-------------------------------------------------------------");
+ e.printStackTrace();
+ }
+ }
+
+ if (!success)
+ throw new RuntimeException("Failed to process some of the specified files");
+ }
+}
diff --git a/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java
new file mode 100644
index 000000000..3a2cd108f
--- /dev/null
+++ b/modules/cassandra-ext/store/src/main/java/org/apache/ignite/cache/store/cassandra/utils/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Contains utility classes
+ */
+
+package org.apache.ignite.cache.store.cassandra.utils;
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/README.txt b/modules/cassandra-ext/store/src/test/bootstrap/aws/README.txt
new file mode 100644
index 000000000..a61b23513
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/README.txt
@@ -0,0 +1,13 @@
+Shell scripts to spin up Ignite, Cassandra and Load tests clusters in AWS.
+
+1) cassandra - bootstrap scripts for Cassandra cluster nodes
+2) ganglia - bootstrap scripts for Ganglia master and agents
+3) ignite - bootstrap scripts for Ignite cluster nodes
+4) tests - bootstrap scripts for Load Tests cluster nodes
+5) common.sh - definitions for common functions
+6) env.sh - definitions for common variables
+7) log-collector.sh - log collector daemon script, to collect logs and upload them to S3
+
+For more details please look at the documentation:
+
+ https://apacheignite.readme.io/docs/aws-infrastructure-deployment
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh
new file mode 100644
index 000000000..017b1b13b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-bootstrap.sh
@@ -0,0 +1,336 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Bootstrap script to spin up Cassandra cluster
+# -----------------------------------------------------------------------------------------------
+
+# URL to download AWS CLI tools
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+# URL to download JDK
+JDK_DOWNLOAD_URL=http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz
+
+# URL to download Ignite-Cassandra tests package - you should previously package and upload it to this place
+TESTS_PACKAGE_DONLOAD_URL=s3:////ignite-cassandra-tests-.zip
+
+# Terminates script execution and upload logs to S3
+terminate()
+{
+ SUCCESS_URL=$S3_CASSANDRA_BOOTSTRAP_SUCCESS
+ FAILURE_URL=$S3_CASSANDRA_BOOTSTRAP_FAILURE
+
+ if [ -n "$SUCCESS_URL" ] && [[ "$SUCCESS_URL" != */ ]]; then
+ SUCCESS_URL=${SUCCESS_URL}/
+ fi
+
+ if [ -n "$FAILURE_URL" ] && [[ "$FAILURE_URL" != */ ]]; then
+ FAILURE_URL=${FAILURE_URL}/
+ fi
+
+ host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ msg=$host_name
+
+ if [ -n "$1" ]; then
+ echo "[ERROR] $1"
+ echo "[ERROR]-----------------------------------------------------"
+ echo "[ERROR] Cassandra node bootstrap failed"
+ echo "[ERROR]-----------------------------------------------------"
+ msg=$1
+
+ if [ -z "$FAILURE_URL" ]; then
+ exit 1
+ fi
+
+ reportFolder=${FAILURE_URL}${host_name}
+ reportFile=$reportFolder/__error__
+ else
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Cassandra node bootstrap successfully completed"
+ echo "[INFO]-----------------------------------------------------"
+
+ if [ -z "$SUCCESS_URL" ]; then
+ exit 0
+ fi
+
+ reportFolder=${SUCCESS_URL}${host_name}
+ reportFile=$reportFolder/__success__
+ fi
+
+ echo $msg > /opt/bootstrap-result
+
+ aws s3 rm --recursive $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop report folder: $reportFolder"
+ fi
+
+ aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+ fi
+
+ rm -f /opt/bootstrap-result
+
+ if [ -n "$1" ]; then
+ exit 1
+ fi
+
+ exit 0
+}
+
+# Downloads specified package
+downloadPackage()
+{
+ echo "[INFO] Downloading $3 package from $1 into $2"
+
+ for i in 0 9;
+ do
+ if [[ "$1" == s3* ]]; then
+ aws s3 cp $1 $2
+ code=$?
+ else
+ curl "$1" -o "$2"
+ code=$?
+ fi
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] $3 package successfully downloaded from $1 into $2"
+ return 0
+ fi
+
+ echo "[WARN] Failed to download $3 package from $i attempt, sleeping extra 5sec"
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to download $3 package from $1 are failed"
+}
+
+# Downloads and setup JDK
+setupJava()
+{
+ rm -Rf /opt/java /opt/jdk.tar.gz
+
+ echo "[INFO] Downloading 'jdk'"
+ wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "$JDK_DOWNLOAD_URL" -O /opt/jdk.tar.gz
+ if [ $? -ne 0 ]; then
+ terminate "Failed to download 'jdk'"
+ fi
+
+ echo "[INFO] Untaring 'jdk'"
+ tar -xvzf /opt/jdk.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar 'jdk'"
+ fi
+
+ rm -Rf /opt/jdk.tar.gz
+
+ unzipDir=$(ls /opt | grep "jdk")
+ if [ "$unzipDir" != "java" ]; then
+ mv /opt/$unzipDir /opt/java
+ fi
+}
+
+# Downloads and setup AWS CLI
+setupAWSCLI()
+{
+ echo "[INFO] Installing 'awscli'"
+ pip install --upgrade awscli
+ if [ $? -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[ERROR] Failed to install 'awscli' using pip"
+ echo "[INFO] Trying to install awscli using zip archive"
+ echo "[INFO] Downloading awscli zip"
+
+ downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+ echo "[INFO] Unzipping awscli zip"
+ unzip /opt/awscli-bundle.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip awscli zip"
+ fi
+
+ rm -Rf /opt/awscli-bundle.zip
+
+ echo "[INFO] Installing awscli"
+ /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install awscli"
+ fi
+
+ echo "[INFO] Successfully installed awscli from zip archive"
+}
+
+# Setup all the pre-requisites (packages, settings and etc.)
+setupPreRequisites()
+{
+ echo "[INFO] Installing 'wget' package"
+ yum -y install wget
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'wget' package"
+ fi
+
+ echo "[INFO] Installing 'net-tools' package"
+ yum -y install net-tools
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'net-tools' package"
+ fi
+
+ echo "[INFO] Installing 'python' package"
+ yum -y install python
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'python' package"
+ fi
+
+ echo "[INFO] Installing 'unzip' package"
+ yum -y install unzip
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'unzip' package"
+ fi
+
+ downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+ echo "[INFO] Installing 'pip'"
+ python /opt/get-pip.py
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'pip'"
+ fi
+}
+
+# Downloads and setup tests package
+setupTestsPackage()
+{
+ downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/ignite-cassandra-tests.zip" "Tests"
+
+ rm -Rf /opt/ignite-cassandra-tests
+
+ unzip /opt/ignite-cassandra-tests.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip tests package"
+ fi
+
+ rm -f /opt/ignite-cassandra-tests.zip
+
+ unzipDir=$(ls /opt | grep "ignite-cassandra")
+ if [ "$unzipDir" != "ignite-cassandra-tests" ]; then
+ mv /opt/$unzipDir /opt/ignite-cassandra-tests
+ fi
+
+ find /opt/ignite-cassandra-tests -type f -name "*.sh" -exec chmod ug+x {} \;
+
+ . /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "cassandra"
+
+ setupNTP
+
+ echo "[INFO] Starting logs collector daemon"
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ /opt/ignite-cassandra-tests/bootstrap/aws/logs-collector.sh "$S3_LOGS_TRIGGER" "$S3_CASSANDRA_LOGS/$HOST_NAME" "/opt/cassandra/logs" "/opt/cassandra/cassandra-start.log" > /opt/logs-collector.log &
+
+ echo "[INFO] Logs collector daemon started: $!"
+
+ echo "----------------------------------------------------------------------------------------"
+ printInstanceInfo
+ echo "----------------------------------------------------------------------------------------"
+ tagInstance
+ bootstrapGangliaAgent "cassandra" 8641
+}
+
+# Downloads Cassandra package
+downloadCassandra()
+{
+ downloadPackage "$CASSANDRA_DOWNLOAD_URL" "/opt/apache-cassandra.tar.gz" "Cassandra"
+
+ rm -Rf /opt/cassandra
+
+ echo "[INFO] Untaring Cassandra package"
+ tar -xvzf /opt/apache-cassandra.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar Cassandra package"
+ fi
+
+ rm -f /opt/apache-cassandra.tar.gz
+
+ unzipDir=$(ls /opt | grep "cassandra" | grep "apache")
+ if [ "$unzipDir" != "cassandra" ]; then
+ mv /opt/$unzipDir /opt/cassandra
+ fi
+}
+
+# Setups Cassandra
+setupCassandra()
+{
+ echo "[INFO] Creating 'cassandra' group"
+ exists=$(cat /etc/group | grep cassandra)
+ if [ -z "$exists" ]; then
+ groupadd cassandra
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'cassandra' group"
+ fi
+ fi
+
+ echo "[INFO] Creating 'cassandra' user"
+ exists=$(cat /etc/passwd | grep cassandra)
+ if [ -z "$exists" ]; then
+ useradd -g cassandra cassandra
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'cassandra' user"
+ fi
+ fi
+
+ rm -f /opt/cassandra/conf/cassandra-env.sh /opt/cassandra/conf/cassandra-template.yaml
+
+ cp /opt/ignite-cassandra-tests/bootstrap/aws/cassandra/cassandra-env.sh /opt/cassandra/conf
+ cp /opt/ignite-cassandra-tests/bootstrap/aws/cassandra/cassandra-template.yaml /opt/cassandra/conf
+
+ chown -R cassandra:cassandra /opt/cassandra /opt/ignite-cassandra-tests
+
+ createCassandraStorageLayout
+
+ cat /opt/cassandra/conf/cassandra-template.yaml | sed -r "s/\\\$\{CASSANDRA_DATA_DIR\}/$CASSANDRA_DATA_DIR/g" > /opt/cassandra/conf/cassandra-template-1.yaml
+ cat /opt/cassandra/conf/cassandra-template-1.yaml | sed -r "s/\\\$\{CASSANDRA_COMMITLOG_DIR\}/$CASSANDRA_COMMITLOG_DIR/g" > /opt/cassandra/conf/cassandra-template-2.yaml
+ cat /opt/cassandra/conf/cassandra-template-2.yaml | sed -r "s/\\\$\{CASSANDRA_CACHES_DIR\}/$CASSANDRA_CACHES_DIR/g" > /opt/cassandra/conf/cassandra-template-3.yaml
+
+ rm -f /opt/cassandra/conf/cassandra-template.yaml /opt/cassandra/conf/cassandra-template-1.yaml /opt/cassandra/conf/cassandra-template-2.yaml
+ mv /opt/cassandra/conf/cassandra-template-3.yaml /opt/cassandra/conf/cassandra-template.yaml
+
+ echo "export JAVA_HOME=/opt/java" >> $1
+ echo "export CASSANDRA_HOME=/opt/cassandra" >> $1
+ echo "export PATH=\$JAVA_HOME/bin:\$CASSANDRA_HOME/bin:\$PATH" >> $1
+}
+
+###################################################################################################################
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Cassandra node"
+echo "[INFO]-----------------------------------------------------------------"
+
+setupPreRequisites
+setupJava
+setupAWSCLI
+setupTestsPackage
+downloadCassandra
+setupCassandra "/root/.bash_profile"
+
+cmd="/opt/ignite-cassandra-tests/bootstrap/aws/cassandra/cassandra-start.sh"
+
+#sudo -u cassandra -g cassandra sh -c "$cmd | tee /opt/cassandra/cassandra-start.log"
+
+$cmd | tee /opt/cassandra/cassandra-start.log
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-env.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-env.sh
new file mode 100644
index 000000000..ba764018b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-env.sh
@@ -0,0 +1,287 @@
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Environment setup script from Cassandra distribution
+# -----------------------------------------------------------------------------------------------
+
+calculate_heap_sizes()
+{
+ case "`uname`" in
+ Linux)
+ system_memory_in_mb=`free -m | awk '/:/ {print $2;exit}'`
+ system_cpu_cores=`egrep -c 'processor([[:space:]]+):.*' /proc/cpuinfo`
+ ;;
+ FreeBSD)
+ system_memory_in_bytes=`sysctl hw.physmem | awk '{print $2}'`
+ system_memory_in_mb=`expr $system_memory_in_bytes / 1024 / 1024`
+ system_cpu_cores=`sysctl hw.ncpu | awk '{print $2}'`
+ ;;
+ SunOS)
+ system_memory_in_mb=`prtconf | awk '/Memory size:/ {print $3}'`
+ system_cpu_cores=`psrinfo | wc -l`
+ ;;
+ Darwin)
+ system_memory_in_bytes=`sysctl hw.memsize | awk '{print $2}'`
+ system_memory_in_mb=`expr $system_memory_in_bytes / 1024 / 1024`
+ system_cpu_cores=`sysctl hw.ncpu | awk '{print $2}'`
+ ;;
+ *)
+ # assume reasonable defaults for e.g. a modern desktop or
+ # cheap server
+ system_memory_in_mb="2048"
+ system_cpu_cores="2"
+ ;;
+ esac
+
+ # some systems like the raspberry pi don't report cores, use at least 1
+ if [ "$system_cpu_cores" -lt "1" ]
+ then
+ system_cpu_cores="1"
+ fi
+
+ # set max heap size based on the following
+ # max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
+ # calculate 1/2 ram and cap to 1024MB
+ # calculate 1/4 ram and cap to 8192MB
+ # pick the max
+ half_system_memory_in_mb=`expr $system_memory_in_mb / 2`
+ quarter_system_memory_in_mb=`expr $half_system_memory_in_mb / 2`
+ if [ "$half_system_memory_in_mb" -gt "1024" ]
+ then
+ half_system_memory_in_mb="1024"
+ fi
+ if [ "$quarter_system_memory_in_mb" -gt "8192" ]
+ then
+ quarter_system_memory_in_mb="8192"
+ fi
+ if [ "$half_system_memory_in_mb" -gt "$quarter_system_memory_in_mb" ]
+ then
+ max_heap_size_in_mb="$half_system_memory_in_mb"
+ else
+ max_heap_size_in_mb="$quarter_system_memory_in_mb"
+ fi
+ MAX_HEAP_SIZE="${max_heap_size_in_mb}M"
+
+ # Young gen: min(max_sensible_per_modern_cpu_core * num_cores, 1/4 * heap size)
+ max_sensible_yg_per_core_in_mb="100"
+ max_sensible_yg_in_mb=`expr $max_sensible_yg_per_core_in_mb "*" $system_cpu_cores`
+
+ desired_yg_in_mb=`expr $max_heap_size_in_mb / 4`
+
+ if [ "$desired_yg_in_mb" -gt "$max_sensible_yg_in_mb" ]
+ then
+ HEAP_NEWSIZE="${max_sensible_yg_in_mb}M"
+ else
+ HEAP_NEWSIZE="${desired_yg_in_mb}M"
+ fi
+}
+
+# Determine the sort of JVM we'll be running on.
+java_ver_output=`"${JAVA:-java}" -version 2>&1`
+jvmver=`echo "$java_ver_output" | grep '[openjdk|java] version' | awk -F'"' 'NR==1 {print $2}'`
+JVM_VERSION=${jvmver%_*}
+JVM_PATCH_VERSION=${jvmver#*_}
+
+if [ "$JVM_VERSION" \< "1.8" ] ; then
+ echo "Cassandra 3.0 and later require Java 8u40 or later."
+ exit 1;
+fi
+
+if [ "$JVM_VERSION" \< "1.8" ] && [ "$JVM_PATCH_VERSION" \< "40" ] ; then
+ echo "Cassandra 3.0 and later require Java 8u40 or later."
+ exit 1;
+fi
+
+jvm=`echo "$java_ver_output" | grep -A 1 'java version' | awk 'NR==2 {print $1}'`
+case "$jvm" in
+ OpenJDK)
+ JVM_VENDOR=OpenJDK
+ # this will be "64-Bit" or "32-Bit"
+ JVM_ARCH=`echo "$java_ver_output" | awk 'NR==3 {print $2}'`
+ ;;
+ "Java(TM)")
+ JVM_VENDOR=Oracle
+ # this will be "64-Bit" or "32-Bit"
+ JVM_ARCH=`echo "$java_ver_output" | awk 'NR==3 {print $3}'`
+ ;;
+ *)
+ # Help fill in other JVM values
+ JVM_VENDOR=other
+ JVM_ARCH=unknown
+ ;;
+esac
+
+# Override these to set the amount of memory to allocate to the JVM at
+# start-up. For production use you may wish to adjust this for your
+# environment. MAX_HEAP_SIZE is the total amount of memory dedicated
+# to the Java heap. HEAP_NEWSIZE refers to the size of the young
+# generation. Both MAX_HEAP_SIZE and HEAP_NEWSIZE should be either set
+# or not (if you set one, set the other).
+#
+# The main trade-off for the young generation is that the larger it
+# is, the longer GC pause times will be. The shorter it is, the more
+# expensive GC will be (usually).
+#
+# The example HEAP_NEWSIZE assumes a modern 8-core+ machine for decent pause
+# times. If in doubt, and if you do not particularly want to tweak, go with
+# 100 MB per physical CPU core.
+
+#MAX_HEAP_SIZE="4G"
+#HEAP_NEWSIZE="800M"
+
+# Set this to control the amount of arenas per-thread in glibc
+#export MALLOC_ARENA_MAX=4
+
+# only calculate the size if it's not set manually
+if [ "x$MAX_HEAP_SIZE" = "x" ] && [ "x$HEAP_NEWSIZE" = "x" ]; then
+ calculate_heap_sizes
+else
+ if [ "x$MAX_HEAP_SIZE" = "x" ] || [ "x$HEAP_NEWSIZE" = "x" ]; then
+ echo "please set or unset MAX_HEAP_SIZE and HEAP_NEWSIZE in pairs (see cassandra-env.sh)"
+ exit 1
+ fi
+fi
+
+if [ "x$MALLOC_ARENA_MAX" = "x" ] ; then
+ export MALLOC_ARENA_MAX=4
+fi
+
+#GC log path has to be defined here because it needs to access CASSANDRA_HOME
+JVM_OPTS="$JVM_OPTS -Xloggc:${CASSANDRA_HOME}/logs/gc.log"
+
+# Here we create the arguments that will get passed to the jvm when
+# starting cassandra.
+
+# Read user-defined JVM options from jvm.options file
+JVM_OPTS_FILE=$CASSANDRA_CONF/jvm.options
+for opt in `grep "^-" $JVM_OPTS_FILE`
+do
+ JVM_OPTS="$JVM_OPTS $opt"
+done
+
+# Check what parameters were defined on jvm.options file to avoid conflicts
+echo $JVM_OPTS | grep -q Xmn
+DEFINED_XMN=$?
+echo $JVM_OPTS | grep -q Xmx
+DEFINED_XMX=$?
+echo $JVM_OPTS | grep -q Xms
+DEFINED_XMS=$?
+echo $JVM_OPTS | grep -q UseConcMarkSweepGC
+USING_CMS=$?
+
+# We only set -Xms and -Xmx if they were not defined on jvm.options file
+# If defined, both Xmx and Xms should be defined together.
+if [ $DEFINED_XMX -ne 0 ] && [ $DEFINED_XMS -ne 0 ]; then
+ JVM_OPTS="$JVM_OPTS -Xms${MAX_HEAP_SIZE}"
+ JVM_OPTS="$JVM_OPTS -Xmx${MAX_HEAP_SIZE}"
+elif [ $DEFINED_XMX -ne 0 ] || [ $DEFINED_XMS -ne 0 ]; then
+ echo "Please set or unset -Xmx and -Xms flags in pairs on jvm.options file."
+ exit 1
+fi
+
+# We only set -Xmn flag if it was not defined in jvm.options file
+# and if the CMS GC is being used
+# If defined, both Xmn and Xmx should be defined together.
+if [ $DEFINED_XMN -eq 0 ] && [ $DEFINED_XMX -ne 0 ]; then
+ echo "Please set or unset -Xmx and -Xmn flags in pairs on jvm.options file."
+ exit 1
+elif [ $DEFINED_XMN -ne 0 ] && [ $USING_CMS -eq 0 ]; then
+ JVM_OPTS="$JVM_OPTS -Xmn${HEAP_NEWSIZE}"
+fi
+
+if [ "$JVM_ARCH" = "64-Bit" ] && [ $USING_CMS -eq 0 ]; then
+ JVM_OPTS="$JVM_OPTS -XX:+UseCondCardMark"
+fi
+
+# provides hints to the JIT compiler
+JVM_OPTS="$JVM_OPTS -XX:CompileCommandFile=$CASSANDRA_CONF/hotspot_compiler"
+
+# add the jamm javaagent
+JVM_OPTS="$JVM_OPTS -javaagent:$CASSANDRA_HOME/lib/jamm-0.3.0.jar"
+
+# set jvm HeapDumpPath with CASSANDRA_HEAPDUMP_DIR
+if [ "x$CASSANDRA_HEAPDUMP_DIR" != "x" ]; then
+ JVM_OPTS="$JVM_OPTS -XX:HeapDumpPath=$CASSANDRA_HEAPDUMP_DIR/cassandra-`date +%s`-pid$$.hprof"
+fi
+
+# jmx: metrics and administration interface
+#
+# add this if you're having trouble connecting:
+# JVM_OPTS="$JVM_OPTS -Djava.rmi.server.hostname="
+#
+# see
+# https://blogs.oracle.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole
+# for more on configuring JMX through firewalls, etc. (Short version:
+# get it working with no firewall first.)
+#
+# Cassandra ships with JMX accessible *only* from localhost.
+# To enable remote JMX connections, uncomment lines below
+# with authentication and/or ssl enabled. See https://wiki.apache.org/cassandra/JmxSecurity
+#
+if [ "x$LOCAL_JMX" = "x" ]; then
+ LOCAL_JMX=yes
+fi
+
+# Specifies the default port over which Cassandra will be available for
+# JMX connections.
+# For security reasons, you should not expose this port to the internet. Firewall it if needed.
+JMX_PORT="7199"
+
+if [ "$LOCAL_JMX" = "yes" ]; then
+# JVM_OPTS="$JVM_OPTS -Dcassandra.jmx.local.port=$JMX_PORT -XX:+DisableExplicitGC"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.local.only=false"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+ JVM_OPTS="$JVM_OPTS -XX:+UnlockCommercialFeatures"
+ JVM_OPTS="$JVM_OPTS -XX:+FlightRecorder"
+ JVM_OPTS="$JVM_OPTS -XX:FlightRecorderOptions=defaultrecording=true"
+else
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.rmi.port=$JMX_PORT"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=true"
+ JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.password.file=/etc/cassandra/jmxremote.password"
+# JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStore=/path/to/keystore"
+# JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.keyStorePassword="
+# JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStore=/path/to/truststore"
+# JVM_OPTS="$JVM_OPTS -Djavax.net.ssl.trustStorePassword="
+# JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.need.client.auth=true"
+# JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.registry.ssl=true"
+# JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.protocols="
+# JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl.enabled.cipher.suites="
+fi
+
+# To use mx4j, an HTML interface for JMX, add mx4j-tools.jar to the lib/
+# directory.
+# See http://wiki.apache.org/cassandra/Operations#Monitoring_with_MX4J
+# By default mx4j listens on 0.0.0.0:8081. Uncomment the following lines
+# to control its listen address and port.
+#MX4J_ADDRESS="-Dmx4jaddress=127.0.0.1"
+#MX4J_PORT="-Dmx4jport=8081"
+
+# Cassandra uses SIGAR to capture OS metrics CASSANDRA-7838
+# for SIGAR we have to set the java.library.path
+# to the location of the native libraries.
+JVM_OPTS="$JVM_OPTS -Djava.library.path=$CASSANDRA_HOME/lib/sigar-bin"
+
+JVM_OPTS="$JVM_OPTS $MX4J_ADDRESS"
+JVM_OPTS="$JVM_OPTS $MX4J_PORT"
+JVM_OPTS="$JVM_OPTS $JVM_EXTRA_OPTS"
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-start.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-start.sh
new file mode 100644
index 000000000..4a6daef6c
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-start.sh
@@ -0,0 +1,217 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Script to start Cassandra daemon (used by cassandra-bootstrap.sh)
+# -----------------------------------------------------------------------------------------------
+
+#profile=/home/cassandra/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+. /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "cassandra"
+
+# Setups Cassandra seeds for this EC2 node. Looks for the information in S3 about
+# already up and running Cassandra cluster nodes
+setupCassandraSeeds()
+{
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ echo "[INFO] Setting up Cassandra seeds"
+
+ CLUSTER_SEEDS=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Using host address as a seed for the first Cassandra node: $CLUSTER_SEEDS"
+
+ aws s3 rm --recursive ${S3_CASSANDRA_NODES_DISCOVERY::-1}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to clean Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY"
+ fi
+ else
+ setupClusterSeeds "cassandra" "true"
+ CLUSTER_SEEDS=$(echo $CLUSTER_SEEDS | sed -r "s/ /,/g")
+ fi
+
+ cat /opt/cassandra/conf/cassandra-template.yaml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CLUSTER_SEEDS/g" > /opt/cassandra/conf/cassandra.yaml
+}
+
+# Gracefully starts Cassandra daemon and waits until it joins Cassandra cluster
+startCassandra()
+{
+ echo "[INFO]-------------------------------------------------------------"
+ echo "[INFO] Trying attempt $START_ATTEMPT to start Cassandra daemon"
+ echo "[INFO]-------------------------------------------------------------"
+ echo ""
+
+ setupCassandraSeeds
+
+ waitToJoinCluster
+
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ aws s3 rm --recursive ${S3_CASSANDRA_NODES_DISCOVERY::-1}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to clean Cassandra node discovery URL: $S3_IGNITE_NODES_DISCOVERY"
+ fi
+ fi
+
+ proc=$(ps -ef | grep java | grep "org.apache.cassandra.service.CassandraDaemon")
+ proc=($proc)
+
+ if [ -n "${proc[1]}" ]; then
+ echo "[INFO] Terminating existing Cassandra process ${proc[1]}"
+ kill -9 ${proc[1]}
+ fi
+
+ echo "[INFO] Starting Cassandra"
+ rm -Rf /opt/cassandra/logs/* /storage/cassandra/*
+ /opt/cassandra/bin/cassandra -R &
+
+ echo "[INFO] Cassandra job id: $!"
+
+ sleep 1m
+
+ START_ATTEMPT=$(( $START_ATTEMPT+1 ))
+}
+
+#######################################################################################################
+
+START_ATTEMPT=0
+
+# Cleans all the previous metadata about this EC2 node
+unregisterNode
+
+# Tries to get first-node lock
+tryToGetFirstNodeLock
+
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ echo "[INFO] Starting first Cassandra node"
+else
+ echo "[INFO] Starting Cassandra node"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+printInstanceInfo
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE_LOCK" != "true" ]; then
+ waitFirstClusterNodeRegistered "true"
+else
+ cleanupMetadata
+fi
+
+# Start Cassandra daemon
+startCassandra
+
+startTime=$(date +%s)
+
+# Trying multiple attempts to start Cassandra daemon
+while true; do
+ proc=$(ps -ef | grep java | grep "org.apache.cassandra.service.CassandraDaemon")
+
+ /opt/cassandra/bin/nodetool status &> /dev/null
+
+ if [ $? -eq 0 ]; then
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Cassandra daemon successfully started"
+ echo "[INFO]-----------------------------------------------------"
+ echo $proc
+ echo "[INFO]-----------------------------------------------------"
+
+ # Once node joined the cluster we need to remove cluster-join lock
+ # to allow other EC2 nodes to acquire it and join cluster sequentially
+ removeClusterJoinLock
+
+ break
+ fi
+
+ currentTime=$(date +%s)
+ duration=$(( $currentTime-$startTime ))
+ duration=$(( $duration/60 ))
+
+ if [ $duration -gt $SERVICE_STARTUP_TIME ]; then
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ # If the first node of Cassandra cluster failed to start Cassandra daemon in SERVICE_STARTUP_TIME min,
+ # we will not try any other attempts and just terminate with error. Terminate function itself, will
+ # take care about removing all the locks holding by this node.
+ terminate "${SERVICE_STARTUP_TIME}min timeout expired, but first Cassandra daemon is still not up and running"
+ else
+ # If node isn't the first node of Cassandra cluster and it failed to start we need to
+ # remove cluster-join lock to allow other EC2 nodes to acquire it
+ removeClusterJoinLock
+
+ # If node failed all SERVICE_START_ATTEMPTS attempts to start Cassandra daemon we will not
+ # try anymore and terminate with error
+ if [ $START_ATTEMPT -gt $SERVICE_START_ATTEMPTS ]; then
+ terminate "${SERVICE_START_ATTEMPTS} attempts exceed, but Cassandra daemon is still not up and running"
+ fi
+
+ # New attempt to start Cassandra daemon
+ startCassandra
+ fi
+
+ continue
+ fi
+
+ # Checking for the situation when two nodes trying to simultaneously join Cassandra cluster.
+ # This actually can happen only in not standard situation, when you are trying to start
+ # Cassandra daemon on some EC2 nodes manually and not using bootstrap script.
+ concurrencyError=$(cat /opt/cassandra/logs/system.log | grep "java.lang.UnsupportedOperationException: Other bootstrapping/leaving/moving nodes detected, cannot bootstrap while cassandra.consistent.rangemovement is true")
+
+ if [ -n "$concurrencyError" ] && [ "$FIRST_NODE_LOCK" != "true" ]; then
+ # Remove cluster-join lock to allow other EC2 nodes to acquire it
+ removeClusterJoinLock
+
+ echo "[WARN] Failed to concurrently start Cassandra daemon. Sleeping for extra 30sec"
+ sleep 30s
+
+ # New attempt to start Cassandra daemon
+ startCassandra
+
+ continue
+ fi
+
+ # Handling situation when Cassandra daemon process abnormally terminated
+ if [ -z "$proc" ]; then
+ # If this is the first node of Cassandra cluster just terminating with error
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ terminate "Failed to start Cassandra daemon"
+ fi
+
+ # Remove cluster-join lock to allow other EC2 nodes to acquire it
+ removeClusterJoinLock
+
+ echo "[WARN] Failed to start Cassandra daemon. Sleeping for extra 30sec"
+ sleep 30s
+
+ # New attempt to start Cassandra daemon
+ startCassandra
+
+ continue
+ fi
+
+ echo "[INFO] Waiting for Cassandra daemon to start, time passed ${duration}min"
+ sleep 30s
+done
+
+# Once Cassandra daemon successfully started we registering new Cassandra node in S3
+registerNode
+
+# Terminating script with zero exit code
+terminate
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-template.yaml b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-template.yaml
new file mode 100644
index 000000000..e621886d6
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/cassandra/cassandra-template.yaml
@@ -0,0 +1,888 @@
+#
+# 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.
+#
+
+# Cassandra storage config YAML
+
+# NOTE:
+# See http://wiki.apache.org/cassandra/StorageConfiguration for
+# full explanations of configuration directives
+# /NOTE
+
+# The name of the cluster. This is mainly used to prevent machines in
+# one logical cluster from joining another.
+cluster_name: 'CassandraIgnite'
+
+# It makes new (non-seed) nodes automatically migrate the right data to themselves.
+# When initializing a fresh cluster with no data, add auto_bootstrap: false
+auto_bootstrap: false
+
+# This defines the number of tokens randomly assigned to this node on the ring
+# The more tokens, relative to other nodes, the larger the proportion of data
+# that this node will store. You probably want all nodes to have the same number
+# of tokens assuming they have equal hardware capability.
+#
+# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility,
+# and will use the initial_token as described below.
+#
+# Specifying initial_token will override this setting on the node's initial start,
+# on subsequent starts, this setting will apply even if initial token is set.
+#
+# If you already have a cluster with 1 token per node, and wish to migrate to
+# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations
+num_tokens: 256
+
+# initial_token allows you to specify tokens manually. While you can use # it with
+# vnodes (num_tokens > 1, above) -- in which case you should provide a
+# comma-separated list -- it's primarily used when adding nodes # to legacy clusters
+# that do not have vnodes enabled.
+# initial_token:
+
+# See http://wiki.apache.org/cassandra/HintedHandoff
+# May either be "true" or "false" to enable globally, or contain a list
+# of data centers to enable per-datacenter.
+# hinted_handoff_enabled: DC1,DC2
+hinted_handoff_enabled: true
+# this defines the maximum amount of time a dead host will have hints
+# generated. After it has been dead this long, new hints for it will not be
+# created until it has been seen alive and gone down again.
+max_hint_window_in_ms: 10800000 # 3 hours
+# Maximum throttle in KBs per second, per delivery thread. This will be
+# reduced proportionally to the number of nodes in the cluster. (If there
+# are two nodes in the cluster, each delivery thread will use the maximum
+# rate; if there are three, each will throttle to half of the maximum,
+# since we expect two nodes to be delivering hints simultaneously.)
+hinted_handoff_throttle_in_kb: 1024
+# Number of threads with which to deliver hints;
+# Consider increasing this number when you have multi-dc deployments, since
+# cross-dc handoff tends to be slower
+max_hints_delivery_threads: 2
+
+# Maximum throttle in KBs per second, total. This will be
+# reduced proportionally to the number of nodes in the cluster.
+batchlog_replay_throttle_in_kb: 1024
+
+# Authentication backend, implementing IAuthenticator; used to identify users
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator,
+# PasswordAuthenticator}.
+#
+# - AllowAllAuthenticator performs no checks - set it to disable authentication.
+# - PasswordAuthenticator relies on username/password pairs to authenticate
+# users. It keeps usernames and hashed passwords in system_auth.credentials table.
+# Please increase system_auth keyspace replication factor if you use this authenticator.
+# If using PasswordAuthenticator, CassandraRoleManager must also be used (see below)
+#authenticator: PasswordAuthenticator
+authenticator: AllowAllAuthenticator
+
+# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions
+# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer,
+# CassandraAuthorizer}.
+#
+# - AllowAllAuthorizer allows any action to any user - set it to disable authorization.
+# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please
+# increase system_auth keyspace replication factor if you use this authorizer.
+#authorizer: CassandraAuthorizer
+authorizer: AllowAllAuthorizer
+
+# Part of the Authentication & Authorization backend, implementing IRoleManager; used
+# to maintain grants and memberships between roles.
+# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager,
+# which stores role information in the system_auth keyspace. Most functions of the
+# IRoleManager require an authenticated login, so unless the configured IAuthenticator
+# actually implements authentication, most of this functionality will be unavailable.
+#
+# - CassandraRoleManager stores role data in the system_auth keyspace. Please
+# increase system_auth keyspace replication factor if you use this role manager.
+role_manager: CassandraRoleManager
+
+# Validity period for roles cache (fetching permissions can be an
+# expensive operation depending on the authorizer). Granted roles are cached for
+# authenticated sessions in AuthenticatedUser and after the period specified
+# here, become eligible for (async) reload.
+# Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthenticator.
+roles_validity_in_ms: 2000
+
+# Refresh interval for roles cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If roles_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as roles_validity_in_ms.
+# roles_update_interval_in_ms: 1000
+
+# Validity period for permissions cache (fetching permissions can be an
+# expensive operation depending on the authorizer, CassandraAuthorizer is
+# one example). Defaults to 2000, set to 0 to disable.
+# Will be disabled automatically for AllowAllAuthorizer.
+permissions_validity_in_ms: 2000
+
+# Refresh interval for permissions cache (if enabled).
+# After this interval, cache entries become eligible for refresh. Upon next
+# access, an async reload is scheduled and the old value returned until it
+# completes. If permissions_validity_in_ms is non-zero, then this must be
+# also.
+# Defaults to the same value as permissions_validity_in_ms.
+# permissions_update_interval_in_ms: 1000
+
+# The partitioner is responsible for distributing groups of rows (by
+# partition key) across nodes in the cluster. You should leave this
+# alone for new clusters. The partitioner can NOT be changed without
+# reloading all data, so when upgrading you should set this to the
+# same partitioner you were already using.
+#
+# Besides Murmur3Partitioner, partitioners included for backwards
+# compatibility include RandomPartitioner, ByteOrderedPartitioner, and
+# OrderPreservingPartitioner.
+#
+partitioner: org.apache.cassandra.dht.Murmur3Partitioner
+
+# Directories where Cassandra should store data on disk. Cassandra
+# will spread data evenly across them, subject to the granularity of
+# the configured compaction strategy.
+# If not set, the default directory is $CASSANDRA_HOME/data/data.
+data_file_directories: ${CASSANDRA_DATA_DIR}
+
+# commit log. when running on magnetic HDD, this should be a
+# separate spindle than the data directories.
+# If not set, the default directory is $CASSANDRA_HOME/data/commitlog.
+commitlog_directory: ${CASSANDRA_COMMITLOG_DIR}
+
+# policy for data disk failures:
+# die: shut down gossip and client transports and kill the JVM for any fs errors or
+# single-sstable errors, so the node can be replaced.
+# stop_paranoid: shut down gossip and client transports even for single-sstable errors,
+# kill the JVM for errors during startup.
+# stop: shut down gossip and client transports, leaving the node effectively dead, but
+# can still be inspected via JMX, kill the JVM for errors during startup.
+# best_effort: stop using the failed disk and respond to requests based on
+# remaining available sstables. This means you WILL see obsolete
+# data at CL.ONE!
+# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra
+disk_failure_policy: stop
+
+# policy for commit disk failures:
+# die: shut down gossip and Thrift and kill the JVM, so the node can be replaced.
+# stop: shut down gossip and Thrift, leaving the node effectively dead, but
+# can still be inspected via JMX.
+# stop_commit: shutdown the commit log, letting writes collect but
+# continuing to service reads, as in pre-2.0.5 Cassandra
+# ignore: ignore fatal errors and let the batches fail
+commit_failure_policy: stop
+
+# Maximum size of the key cache in memory.
+#
+# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the
+# minimum, sometimes more. The key cache is fairly tiny for the amount of
+# time it saves, so it's worthwhile to use it at large numbers.
+# The row cache saves even more time, but must contain the entire row,
+# so it is extremely space-intensive. It's best to only use the
+# row cache if you have hot rows or static rows.
+#
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache.
+key_cache_size_in_mb:
+
+# Duration in seconds after which Cassandra should
+# save the key cache. Caches are saved to saved_caches_directory as
+# specified in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 14400 or 4 hours.
+key_cache_save_period: 14400
+
+# Number of keys from the key cache to save
+# Disabled by default, meaning all keys are going to be saved
+# key_cache_keys_to_save: 100
+
+# Row cache implementation class name.
+# Available implementations:
+# org.apache.cassandra.cache.OHCProvider Fully off-heap row cache implementation (default).
+# org.apache.cassandra.cache.SerializingCacheProvider This is the row cache implementation availabile
+# in previous releases of Cassandra.
+# row_cache_class_name: org.apache.cassandra.cache.OHCProvider
+
+# Maximum size of the row cache in memory.
+# Please note that OHC cache implementation requires some additional off-heap memory to manage
+# the map structures and some in-flight memory during operations before/after cache entries can be
+# accounted against the cache capacity. This overhead is usually small compared to the whole capacity.
+# Do not specify more memory that the system can afford in the worst usual situation and leave some
+# headroom for OS block level cache. Do never allow your system to swap.
+#
+# Default value is 0, to disable row caching.
+row_cache_size_in_mb: 0
+
+# Duration in seconds after which Cassandra should save the row cache.
+# Caches are saved to saved_caches_directory as specified in this configuration file.
+#
+# Saved caches greatly improve cold-start speeds, and is relatively cheap in
+# terms of I/O for the key cache. Row cache saving is much more expensive and
+# has limited use.
+#
+# Default is 0 to disable saving the row cache.
+row_cache_save_period: 0
+
+# Number of keys from the row cache to save.
+# Specify 0 (which is the default), meaning all keys are going to be saved
+# row_cache_keys_to_save: 100
+
+# Maximum size of the counter cache in memory.
+#
+# Counter cache helps to reduce counter locks' contention for hot counter cells.
+# In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before
+# write entirely. With RF > 1 a counter cache hit will still help to reduce the duration
+# of the lock hold, helping with hot counter cell updates, but will not allow skipping
+# the read entirely. Only the local (clock, count) tuple of a counter cell is kept
+# in memory, not the whole counter, so it's relatively cheap.
+#
+# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup.
+#
+# Default value is empty to make it "auto" (min(2.5% of Heap (in MB), 50MB)). Set to 0 to disable counter cache.
+# NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache.
+counter_cache_size_in_mb:
+
+# Duration in seconds after which Cassandra should
+# save the counter cache (keys only). Caches are saved to saved_caches_directory as
+# specified in this configuration file.
+#
+# Default is 7200 or 2 hours.
+counter_cache_save_period: 7200
+
+# Number of keys from the counter cache to save
+# Disabled by default, meaning all keys are going to be saved
+# counter_cache_keys_to_save: 100
+
+# The off-heap memory allocator. Affects storage engine metadata as
+# well as caches. Experiments show that JEMAlloc saves some memory
+# than the native GCC allocator (i.e., JEMalloc is more
+# fragmentation-resistant).
+#
+# Supported values are: NativeAllocator, JEMallocAllocator
+#
+# If you intend to use JEMallocAllocator you have to install JEMalloc as library and
+# modify cassandra-env.sh as directed in the file.
+#
+# Defaults to NativeAllocator
+# memory_allocator: NativeAllocator
+
+# saved caches
+# If not set, the default directory is $CASSANDRA_HOME/data/saved_caches.
+saved_caches_directory: ${CASSANDRA_CACHES_DIR}
+
+# commitlog_sync may be either "periodic" or "batch."
+#
+# When in batch mode, Cassandra won't ack writes until the commit log
+# has been fsynced to disk. It will wait
+# commitlog_sync_batch_window_in_ms milliseconds between fsyncs.
+# This window should be kept short because the writer threads will
+# be unable to do extra work while waiting. (You may need to increase
+# concurrent_writes for the same reason.)
+#
+# commitlog_sync: batch
+# commitlog_sync_batch_window_in_ms: 2
+#
+# the other option is "periodic" where writes may be acked immediately
+# and the CommitLog is simply synced every commitlog_sync_period_in_ms
+# milliseconds.
+commitlog_sync: periodic
+commitlog_sync_period_in_ms: 10000
+
+# The size of the individual commitlog file segments. A commitlog
+# segment may be archived, deleted, or recycled once all the data
+# in it (potentially from each columnfamily in the system) has been
+# flushed to sstables.
+#
+# The default size is 32, which is almost always fine, but if you are
+# archiving commitlog segments (see commitlog_archiving.properties),
+# then you probably want a finer granularity of archiving; 8 or 16 MB
+# is reasonable.
+commitlog_segment_size_in_mb: 32
+
+# Compression to apply to the commit log. If omitted, the commit log
+# will be written uncompressed. LZ4, Snappy, and Deflate compressors
+# are supported.
+#commitlog_compression:
+# - class_name: LZ4Compressor
+# parameters:
+# -
+
+# any class that implements the SeedProvider interface and has a
+# constructor that takes a Map of parameters will do.
+seed_provider:
+ # Addresses of hosts that are deemed contact points.
+ # Cassandra nodes use this list of hosts to find each other and learn
+ # the topology of the ring. You must change this if you are running
+ # multiple nodes!
+ - class_name: org.apache.cassandra.locator.SimpleSeedProvider
+ parameters:
+ # seeds is actually a comma-delimited list of addresses.
+ # Ex: ",,"
+ - seeds: "${CASSANDRA_SEEDS}"
+
+# For workloads with more data than can fit in memory, Cassandra's
+# bottleneck will be reads that need to fetch data from
+# disk. "concurrent_reads" should be set to (16 * number_of_drives) in
+# order to allow the operations to enqueue low enough in the stack
+# that the OS and drives can reorder them. Same applies to
+# "concurrent_counter_writes", since counter writes read the current
+# values before incrementing and writing them back.
+#
+# On the other hand, since writes are almost never IO bound, the ideal
+# number of "concurrent_writes" is dependent on the number of cores in
+# your system; (8 * number_of_cores) is a good rule of thumb.
+concurrent_reads: 32
+concurrent_writes: 32
+concurrent_counter_writes: 32
+
+# Total memory to use for sstable-reading buffers. Defaults to
+# the smaller of 1/4 of heap or 512MB.
+# file_cache_size_in_mb: 512
+
+# Total permitted memory to use for memtables. Cassandra will stop
+# accepting writes when the limit is exceeded until a flush completes,
+# and will trigger a flush based on memtable_cleanup_threshold
+# If omitted, Cassandra will set both to 1/4 the size of the heap.
+# memtable_heap_space_in_mb: 2048
+# memtable_offheap_space_in_mb: 2048
+
+# Ratio of occupied non-flushing memtable size to total permitted size
+# that will trigger a flush of the largest memtable. Lager mct will
+# mean larger flushes and hence less compaction, but also less concurrent
+# flush activity which can make it difficult to keep your disks fed
+# under heavy write load.
+#
+# memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1)
+# memtable_cleanup_threshold: 0.11
+
+# Specify the way Cassandra allocates and manages memtable memory.
+# Options are:
+# heap_buffers: on heap nio buffers
+# offheap_buffers: off heap (direct) nio buffers
+# offheap_objects: native memory, eliminating nio buffer heap overhead
+memtable_allocation_type: heap_buffers
+
+# Total space to use for commit logs on disk.
+#
+# If space gets above this value, Cassandra will flush every dirty CF
+# in the oldest segment and remove it. So a small total commitlog space
+# will tend to cause more flush activity on less-active columnfamilies.
+#
+# The default value is 8192.
+# commitlog_total_space_in_mb: 8192
+
+# This sets the amount of memtable flush writer threads. These will
+# be blocked by disk io, and each one will hold a memtable in memory
+# while blocked.
+#
+# memtable_flush_writers defaults to the smaller of (number of disks,
+# number of cores), with a minimum of 2 and a maximum of 8.
+#
+# If your data directories are backed by SSD, you should increase this
+# to the number of cores.
+#memtable_flush_writers: 8
+
+# A fixed memory pool size in MB for for SSTable index summaries. If left
+# empty, this will default to 5% of the heap size. If the memory usage of
+# all index summaries exceeds this limit, SSTables with low read rates will
+# shrink their index summaries in order to meet this limit. However, this
+# is a best-effort process. In extreme conditions Cassandra may need to use
+# more than this amount of memory.
+index_summary_capacity_in_mb:
+
+# How frequently index summaries should be resampled. This is done
+# periodically to redistribute memory from the fixed-size pool to sstables
+# proportional their recent read rates. Setting to -1 will disable this
+# process, leaving existing index summaries at their current sampling level.
+index_summary_resize_interval_in_minutes: 60
+
+# Whether to, when doing sequential writing, fsync() at intervals in
+# order to force the operating system to flush the dirty
+# buffers. Enable this to avoid sudden dirty buffer flushing from
+# impacting read latencies. Almost always a good idea on SSDs; not
+# necessarily on platters.
+trickle_fsync: false
+trickle_fsync_interval_in_kb: 10240
+
+# TCP port, for commands and data
+# For security reasons, you should not expose this port to the internet. Firewall it if needed.
+storage_port: 7000
+
+# SSL port, for encrypted communication. Unused unless enabled in
+# encryption_options
+# For security reasons, you should not expose this port to the internet. Firewall it if needed.
+ssl_storage_port: 7001
+
+# Address or interface to bind to and tell other Cassandra nodes to connect to.
+# You _must_ change this if you want multiple nodes to be able to communicate!
+#
+# Set listen_address OR listen_interface, not both. Interfaces must correspond
+# to a single address, IP aliasing is not supported.
+#
+# Leaving it blank leaves it up to InetAddress.getLocalHost(). This
+# will always do the Right Thing _if_ the node is properly configured
+# (hostname, name resolution, etc), and the Right Thing is to use the
+# address associated with the hostname (it might not be).
+#
+# Setting listen_address to 0.0.0.0 is always wrong.
+#
+# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address
+# you can specify which should be chosen using listen_interface_prefer_ipv6. If false the first ipv4
+# address will be used. If true the first ipv6 address will be used. Defaults to false preferring
+# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6.
+listen_address:
+# listen_interface: eth0
+# listen_interface_prefer_ipv6: false
+
+# Address to broadcast to other Cassandra nodes
+# Leaving this blank will set it to the same value as listen_address
+# broadcast_address: 1.2.3.4
+
+# Internode authentication backend, implementing IInternodeAuthenticator;
+# used to allow/disallow connections from peer nodes.
+# internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator
+
+# Whether to start the native transport server.
+# Please note that the address on which the native transport is bound is the
+# same as the rpc_address. The port however is different and specified below.
+start_native_transport: true
+# port for the CQL native transport to listen for clients on
+# For security reasons, you should not expose this port to the internet. Firewall it if needed.
+native_transport_port: 9042
+# The maximum threads for handling requests when the native transport is used.
+# This is similar to rpc_max_threads though the default differs slightly (and
+# there is no native_transport_min_threads, idle threads will always be stopped
+# after 30 seconds).
+# native_transport_max_threads: 128
+#
+# The maximum size of allowed frame. Frame (requests) larger than this will
+# be rejected as invalid. The default is 256MB.
+# native_transport_max_frame_size_in_mb: 256
+
+# The maximum number of concurrent client connections.
+# The default is -1, which means unlimited.
+# native_transport_max_concurrent_connections: -1
+
+# The maximum number of concurrent client connections per source ip.
+# The default is -1, which means unlimited.
+# native_transport_max_concurrent_connections_per_ip: -1
+
+# Whether to start the thrift rpc server.
+start_rpc: true
+
+# The address or interface to bind the Thrift RPC service and native transport
+# server to.
+#
+# Set rpc_address OR rpc_interface, not both. Interfaces must correspond
+# to a single address, IP aliasing is not supported.
+#
+# Leaving rpc_address blank has the same effect as on listen_address
+# (i.e. it will be based on the configured hostname of the node).
+#
+# Note that unlike listen_address, you can specify 0.0.0.0, but you must also
+# set broadcast_rpc_address to a value other than 0.0.0.0.
+#
+# For security reasons, you should not expose this port to the internet. Firewall it if needed.
+#
+# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address
+# you can specify which should be chosen using rpc_interface_prefer_ipv6. If false the first ipv4
+# address will be used. If true the first ipv6 address will be used. Defaults to false preferring
+# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6.
+rpc_address:
+# rpc_interface: eth1
+# rpc_interface_prefer_ipv6: false
+
+# port for Thrift to listen for clients on
+rpc_port: 9160
+
+# RPC address to broadcast to drivers and other Cassandra nodes. This cannot
+# be set to 0.0.0.0. If left blank, this will be set to the value of
+# rpc_address. If rpc_address is set to 0.0.0.0, broadcast_rpc_address must
+# be set.
+broadcast_rpc_address:
+
+# enable or disable keepalive on rpc/native connections
+rpc_keepalive: true
+
+# Cassandra provides two out-of-the-box options for the RPC Server:
+#
+# sync -> One thread per thrift connection. For a very large number of clients, memory
+# will be your limiting factor. On a 64 bit JVM, 180KB is the minimum stack size
+# per thread, and that will correspond to your use of virtual memory (but physical memory
+# may be limited depending on use of stack space).
+#
+# hsha -> Stands for "half synchronous, half asynchronous." All thrift clients are handled
+# asynchronously using a small number of threads that does not vary with the amount
+# of thrift clients (and thus scales well to many clients). The rpc requests are still
+# synchronous (one thread per active request). If hsha is selected then it is essential
+# that rpc_max_threads is changed from the default value of unlimited.
+#
+# The default is sync because on Windows hsha is about 30% slower. On Linux,
+# sync/hsha performance is about the same, with hsha of course using less memory.
+#
+# Alternatively, can provide your own RPC server by providing the fully-qualified class name
+# of an o.a.c.t.TServerFactory that can create an instance of it.
+rpc_server_type: sync
+
+# Uncomment rpc_min|max_thread to set request pool size limits.
+#
+# Regardless of your choice of RPC server (see above), the number of maximum requests in the
+# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync
+# RPC server, it also dictates the number of clients that can be connected at all).
+#
+# The default is unlimited and thus provides no protection against clients overwhelming the server. You are
+# encouraged to set a maximum that makes sense for you in production, but do keep in mind that
+# rpc_max_threads represents the maximum number of client requests this server may execute concurrently.
+#
+# rpc_min_threads: 16
+# rpc_max_threads: 2048
+
+# uncomment to set socket buffer sizes on rpc connections
+# rpc_send_buff_size_in_bytes:
+# rpc_recv_buff_size_in_bytes:
+
+# Uncomment to set socket buffer size for internode communication
+# Note that when setting this, the buffer size is limited by net.core.wmem_max
+# and when not setting it it is defined by net.ipv4.tcp_wmem
+# See:
+# /proc/sys/net/core/wmem_max
+# /proc/sys/net/core/rmem_max
+# /proc/sys/net/ipv4/tcp_wmem
+# /proc/sys/net/ipv4/tcp_wmem
+# and: man tcp
+# internode_send_buff_size_in_bytes:
+# internode_recv_buff_size_in_bytes:
+
+# Frame size for thrift (maximum message length).
+thrift_framed_transport_size_in_mb: 15
+
+# Set to true to have Cassandra create a hard link to each sstable
+# flushed or streamed locally in a backups/ subdirectory of the
+# keyspace data. Removing these links is the operator's
+# responsibility.
+incremental_backups: false
+
+# Whether or not to take a snapshot before each compaction. Be
+# careful using this option, since Cassandra won't clean up the
+# snapshots for you. Mostly useful if you're paranoid when there
+# is a data format change.
+snapshot_before_compaction: false
+
+# Whether or not a snapshot is taken of the data before keyspace truncation
+# or dropping of column families. The STRONGLY advised default of true
+# should be used to provide data safety. If you set this flag to false, you will
+# lose data on truncation or drop.
+auto_snapshot: true
+
+# When executing a scan, within or across a partition, we need to keep the
+# tombstones seen in memory so we can return them to the coordinator, which
+# will use them to make sure other replicas also know about the deleted rows.
+# With workloads that generate a lot of tombstones, this can cause performance
+# problems and even exaust the server heap.
+# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets)
+# Adjust the thresholds here if you understand the dangers and want to
+# scan more tombstones anyway. These thresholds may also be adjusted at runtime
+# using the StorageService mbean.
+tombstone_warn_threshold: 1000
+tombstone_failure_threshold: 100000
+
+# Granularity of the collation index of rows within a partition.
+# Increase if your rows are large, or if you have a very large
+# number of rows per partition. The competing goals are these:
+# 1) a smaller granularity means more index entries are generated
+# and looking up rows withing the partition by collation column
+# is faster
+# 2) but, Cassandra will keep the collation index in memory for hot
+# rows (as part of the key cache), so a larger granularity means
+# you can cache more hot rows
+column_index_size_in_kb: 64
+
+
+# Log WARN on any batch size exceeding this value. 5kb per batch by default.
+# Caution should be taken on increasing the size of this threshold as it can lead to node instability.
+batch_size_warn_threshold_in_kb: 5
+
+# Fail any batch exceeding this value. 50kb (10x warn threshold) by default.
+batch_size_fail_threshold_in_kb: 50
+
+# Number of simultaneous compactions to allow, NOT including
+# validation "compactions" for anti-entropy repair. Simultaneous
+# compactions can help preserve read performance in a mixed read/write
+# workload, by mitigating the tendency of small sstables to accumulate
+# during a single long running compactions. The default is usually
+# fine and if you experience problems with compaction running too
+# slowly or too fast, you should look at
+# compaction_throughput_mb_per_sec first.
+#
+# concurrent_compactors defaults to the smaller of (number of disks,
+# number of cores), with a minimum of 2 and a maximum of 8.
+#
+# If your data directories are backed by SSD, you should increase this
+# to the number of cores.
+#concurrent_compactors: 1
+
+# Throttles compaction to the given total throughput across the entire
+# system. The faster you insert data, the faster you need to compact in
+# order to keep the sstable count down, but in general, setting this to
+# 16 to 32 times the rate you are inserting data is more than sufficient.
+# Setting this to 0 disables throttling. Note that this account for all types
+# of compaction, including validation compaction.
+compaction_throughput_mb_per_sec: 16
+
+# Log a warning when compacting partitions larger than this value
+compaction_large_partition_warning_threshold_mb: 100
+
+# When compacting, the replacement sstable(s) can be opened before they
+# are completely written, and used in place of the prior sstables for
+# any range that has been written. This helps to smoothly transfer reads
+# between the sstables, reducing page cache churn and keeping hot rows hot
+sstable_preemptive_open_interval_in_mb: 50
+
+# Throttles all outbound streaming file transfers on this node to the
+# given total throughput in Mbps. This is necessary because Cassandra does
+# mostly sequential IO when streaming data during bootstrap or repair, which
+# can lead to saturating the network connection and degrading rpc performance.
+# When unset, the default is 200 Mbps or 25 MB/s.
+# stream_throughput_outbound_megabits_per_sec: 200
+
+# Throttles all streaming file transfer between the datacenters,
+# this setting allows users to throttle inter dc stream throughput in addition
+# to throttling all network stream traffic as configured with
+# stream_throughput_outbound_megabits_per_sec
+# inter_dc_stream_throughput_outbound_megabits_per_sec:
+
+# How long the coordinator should wait for read operations to complete
+read_request_timeout_in_ms: 50000
+# How long the coordinator should wait for seq or index scans to complete
+range_request_timeout_in_ms: 10000
+# How long the coordinator should wait for writes to complete
+write_request_timeout_in_ms: 20000
+# How long the coordinator should wait for counter writes to complete
+counter_write_request_timeout_in_ms: 5000
+# How long a coordinator should continue to retry a CAS operation
+# that contends with other proposals for the same row
+cas_contention_timeout_in_ms: 1000
+# How long the coordinator should wait for truncates to complete
+# (This can be much longer, because unless auto_snapshot is disabled
+# we need to flush first so we can snapshot before removing the data.)
+truncate_request_timeout_in_ms: 60000
+# The default timeout for other, miscellaneous operations
+request_timeout_in_ms: 10000
+
+# Enable operation timeout information exchange between nodes to accurately
+# measure request timeouts. If disabled, replicas will assume that requests
+# were forwarded to them instantly by the coordinator, which means that
+# under overload conditions we will waste that much extra time processing
+# already-timed-out requests.
+#
+# Warning: before enabling this property make sure to ntp is installed
+# and the times are synchronized between the nodes.
+cross_node_timeout: false
+
+# Enable socket timeout for streaming operation.
+# When a timeout occurs during streaming, streaming is retried from the start
+# of the current file. This _can_ involve re-streaming an important amount of
+# data, so you should avoid setting the value too low.
+# Default value is 3600000, which means streams timeout after an hour.
+# streaming_socket_timeout_in_ms: 3600000
+
+# phi value that must be reached for a host to be marked down.
+# most users should never need to adjust this.
+# phi_convict_threshold: 8
+
+# endpoint_snitch -- Set this to a class that implements
+# IEndpointSnitch. The snitch has two functions:
+# - it teaches Cassandra enough about your network topology to route
+# requests efficiently
+# - it allows Cassandra to spread replicas around your cluster to avoid
+# correlated failures. It does this by grouping machines into
+# "datacenters" and "racks." Cassandra will do its best not to have
+# more than one replica on the same "rack" (which may not actually
+# be a physical location)
+#
+# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER,
+# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS
+# ARE PLACED.
+#
+# Out of the box, Cassandra provides
+# - SimpleSnitch:
+# Treats Strategy order as proximity. This can improve cache
+# locality when disabling read repair. Only appropriate for
+# single-datacenter deployments.
+# - GossipingPropertyFileSnitch
+# This should be your go-to snitch for production use. The rack
+# and datacenter for the local node are defined in
+# cassandra-rackdc.properties and propagated to other nodes via
+# gossip. If cassandra-topology.properties exists, it is used as a
+# fallback, allowing migration from the PropertyFileSnitch.
+# - PropertyFileSnitch:
+# Proximity is determined by rack and data center, which are
+# explicitly configured in cassandra-topology.properties.
+# - Ec2Snitch:
+# Appropriate for EC2 deployments in a single Region. Loads Region
+# and Availability Zone information from the EC2 API. The Region is
+# treated as the datacenter, and the Availability Zone as the rack.
+# Only private IPs are used, so this will not work across multiple
+# Regions.
+# - Ec2MultiRegionSnitch:
+# Uses public IPs as broadcast_address to allow cross-region
+# connectivity. (Thus, you should set seed addresses to the public
+# IP as well.) You will need to open the storage_port or
+# ssl_storage_port on the public IP firewall. (For intra-Region
+# traffic, Cassandra will switch to the private IP after
+# establishing a connection.)
+# - RackInferringSnitch:
+# Proximity is determined by rack and data center, which are
+# assumed to correspond to the 3rd and 2nd octet of each node's IP
+# address, respectively. Unless this happens to match your
+# deployment conventions, this is best used as an example of
+# writing a custom Snitch class and is provided in that spirit.
+#
+# You can use a custom Snitch by setting this to the full class name
+# of the snitch, which will be assumed to be on your classpath.
+endpoint_snitch: Ec2Snitch
+
+# controls how often to perform the more expensive part of host score
+# calculation
+dynamic_snitch_update_interval_in_ms: 100
+# controls how often to reset all host scores, allowing a bad host to
+# possibly recover
+dynamic_snitch_reset_interval_in_ms: 600000
+# if set greater than zero and read_repair_chance is < 1.0, this will allow
+# 'pinning' of replicas to hosts in order to increase cache capacity.
+# The badness threshold will control how much worse the pinned host has to be
+# before the dynamic snitch will prefer other replicas over it. This is
+# expressed as a double which represents a percentage. Thus, a value of
+# 0.2 means Cassandra would continue to prefer the static snitch values
+# until the pinned host was 20% worse than the fastest.
+dynamic_snitch_badness_threshold: 0.1
+
+# request_scheduler -- Set this to a class that implements
+# RequestScheduler, which will schedule incoming client requests
+# according to the specific policy. This is useful for multi-tenancy
+# with a single Cassandra cluster.
+# NOTE: This is specifically for requests from the client and does
+# not affect inter node communication.
+# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place
+# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of
+# client requests to a node with a separate queue for each
+# request_scheduler_id. The scheduler is further customized by
+# request_scheduler_options as described below.
+request_scheduler: org.apache.cassandra.scheduler.NoScheduler
+
+# Scheduler Options vary based on the type of scheduler
+# NoScheduler - Has no options
+# RoundRobin
+# - throttle_limit -- The throttle_limit is the number of in-flight
+# requests per client. Requests beyond
+# that limit are queued up until
+# running requests can complete.
+# The value of 80 here is twice the number of
+# concurrent_reads + concurrent_writes.
+# - default_weight -- default_weight is optional and allows for
+# overriding the default which is 1.
+# - weights -- Weights are optional and will default to 1 or the
+# overridden default_weight. The weight translates into how
+# many requests are handled during each turn of the
+# RoundRobin, based on the scheduler id.
+#
+# request_scheduler_options:
+# throttle_limit: 80
+# default_weight: 5
+# weights:
+# Keyspace1: 1
+# Keyspace2: 5
+
+# request_scheduler_id -- An identifier based on which to perform
+# the request scheduling. Currently the only valid option is keyspace.
+# request_scheduler_id: keyspace
+
+# Enable or disable inter-node encryption
+# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that
+# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher
+# suite for authentication, key exchange and encryption of the actual data transfers.
+# Use the DHE/ECDHE ciphers if running in FIPS 140 compliant mode.
+# NOTE: No custom encryption options are enabled at the moment
+# The available internode options are : all, none, dc, rack
+#
+# If set to dc cassandra will encrypt the traffic between the DCs
+# If set to rack cassandra will encrypt the traffic between the racks
+#
+# The passwords used in these options must match the passwords used when generating
+# the keystore and truststore. For instructions on generating these files, see:
+# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore
+#
+server_encryption_options:
+ internode_encryption: none
+ keystore: conf/.keystore
+ keystore_password: cassandra
+ truststore: conf/.truststore
+ truststore_password: cassandra
+ # More advanced defaults below:
+ # protocol: TLS
+ # algorithm: SunX509
+ # store_type: JKS
+ # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
+ # require_client_auth: false
+
+# enable or disable client/server encryption.
+client_encryption_options:
+ enabled: false
+ keystore: conf/.keystore
+ keystore_password: cassandra
+ # require_client_auth: false
+ # Set trustore and truststore_password if require_client_auth is true
+ # truststore: conf/.truststore
+ # truststore_password: cassandra
+ # More advanced defaults below:
+ # protocol: TLS
+ # algorithm: SunX509
+ # store_type: JKS
+ # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA]
+
+# internode_compression controls whether traffic between nodes is
+# compressed.
+# can be: all - all traffic is compressed
+# dc - traffic between different datacenters is compressed
+# none - nothing is compressed.
+internode_compression: all
+
+# Enable or disable tcp_nodelay for inter-dc communication.
+# Disabling it will result in larger (but fewer) network packets being sent,
+# reducing overhead from the TCP protocol itself, at the cost of increasing
+# latency if you block for cross-datacenter responses.
+inter_dc_tcp_nodelay: false
+
+# TTL for different trace types used during logging of the repair process.
+tracetype_query_ttl: 86400
+tracetype_repair_ttl: 604800
+
+# UDFs (user defined functions) are disabled by default.
+# As of Cassandra 2.2, there is no security manager or anything else in place that
+# prevents execution of evil code. CASSANDRA-9402 will fix this issue for Cassandra 3.0.
+# This will inherently be backwards-incompatible with any 2.2 UDF that perform insecure
+# operations such as opening a socket or writing to the filesystem.
+enable_user_defined_functions: false
+
+# The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation.
+# Lowering this value on Windows can provide much tighter latency and better throughput, however
+# some virtualized environments may see a negative performance impact from changing this setting
+# below their system default. The sysinternals 'clockres' tool can confirm your system's default
+# setting.
+windows_timer_interval: 1
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/common.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/common.sh
new file mode 100644
index 000000000..6469e951c
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/common.sh
@@ -0,0 +1,1481 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Common purpose functions used by bootstrap scripts
+# -----------------------------------------------------------------------------------------------
+
+# Validates values of the main environment variables specified in env.sh
+validate()
+{
+ if [ -n "$TESTS_TYPE" ] && [ "$TESTS_TYPE" != "ignite" ] && [ "$TESTS_TYPE" != "cassandra" ]; then
+ terminate "Incorrect tests type specified: $TESTS_TYPE"
+ fi
+
+ if [ -z "$S3_TESTS_NODES_DISCOVERY" ]; then
+ terminate "Tests discovery URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_NODES_DISCOVERY" != */ ]]; then
+ S3_TESTS_NODES_DISCOVERY=${S3_TESTS_NODES_DISCOVERY}/
+ fi
+
+ if [ -z "$S3_TESTS_SUCCESS" ]; then
+ terminate "Tests success URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_SUCCESS" != */ ]]; then
+ S3_TESTS_SUCCESS=${S3_TESTS_SUCCESS}/
+ fi
+
+ if [ -z "$S3_TESTS_FAILURE" ]; then
+ terminate "Tests failure URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_FAILURE" != */ ]]; then
+ S3_TESTS_FAILURE=${S3_TESTS_FAILURE}/
+ fi
+
+ if [ -z "$S3_TESTS_IDLE" ]; then
+ terminate "Tests idle URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_IDLE" != */ ]]; then
+ S3_TESTS_IDLE=${S3_TESTS_IDLE}/
+ fi
+
+ if [ -z "$S3_TESTS_PREPARING" ]; then
+ terminate "Tests preparing URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_PREPARING" != */ ]]; then
+ S3_TESTS_PREPARING=${S3_TESTS_PREPARING}/
+ fi
+
+ if [ -z "$S3_TESTS_RUNNING" ]; then
+ terminate "Tests running URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_RUNNING" != */ ]]; then
+ S3_TESTS_RUNNING=${S3_TESTS_RUNNING}/
+ fi
+
+ if [ -z "$S3_TESTS_WAITING" ]; then
+ terminate "Tests waiting URL doesn't specified"
+ fi
+
+ if [[ "$S3_TESTS_WAITING" != */ ]]; then
+ S3_TESTS_WAITING=${S3_TESTS_WAITING}/
+ fi
+
+ if [ -z "$S3_IGNITE_NODES_DISCOVERY" ]; then
+ terminate "Ignite discovery URL doesn't specified"
+ fi
+
+ if [[ "$S3_IGNITE_NODES_DISCOVERY" != */ ]]; then
+ S3_IGNITE_NODES_DISCOVERY=${S3_IGNITE_NODES_DISCOVERY}/
+ fi
+
+ if [ -z "$S3_IGNITE_BOOTSTRAP_SUCCESS" ]; then
+ terminate "Ignite success URL doesn't specified"
+ fi
+
+ if [[ "$S3_IGNITE_BOOTSTRAP_SUCCESS" != */ ]]; then
+ S3_IGNITE_BOOTSTRAP_SUCCESS=${S3_IGNITE_BOOTSTRAP_SUCCESS}/
+ fi
+
+ if [ -z "$S3_IGNITE_BOOTSTRAP_FAILURE" ]; then
+ terminate "Ignite failure URL doesn't specified"
+ fi
+
+ if [[ "$S3_IGNITE_BOOTSTRAP_FAILURE" != */ ]]; then
+ S3_IGNITE_BOOTSTRAP_FAILURE=${S3_IGNITE_BOOTSTRAP_FAILURE}/
+ fi
+
+ if [ -z "$S3_CASSANDRA_NODES_DISCOVERY" ]; then
+ terminate "Cassandra discovery URL doesn't specified"
+ fi
+
+ if [[ "$S3_CASSANDRA_NODES_DISCOVERY" != */ ]]; then
+ S3_CASSANDRA_NODES_DISCOVERY=${S3_CASSANDRA_NODES_DISCOVERY}/
+ fi
+
+ if [ -z "$S3_CASSANDRA_BOOTSTRAP_SUCCESS" ]; then
+ terminate "Cassandra success URL doesn't specified"
+ fi
+
+ if [[ "$S3_CASSANDRA_BOOTSTRAP_SUCCESS" != */ ]]; then
+ S3_CASSANDRA_BOOTSTRAP_SUCCESS=${S3_CASSANDRA_BOOTSTRAP_SUCCESS}/
+ fi
+
+ if [ -z "$S3_CASSANDRA_BOOTSTRAP_FAILURE" ]; then
+ terminate "Cassandra failure URL doesn't specified"
+ fi
+
+ if [[ "$S3_CASSANDRA_BOOTSTRAP_FAILURE" != */ ]]; then
+ S3_CASSANDRA_BOOTSTRAP_FAILURE=${S3_CASSANDRA_BOOTSTRAP_FAILURE}/
+ fi
+
+ if [ -z "$S3_GANGLIA_MASTER_DISCOVERY" ]; then
+ terminate "Ganglia master discovery URL doesn't specified"
+ fi
+
+ if [[ "$S3_GANGLIA_MASTER_DISCOVERY" != */ ]]; then
+ S3_GANGLIA_MASTER_DISCOVERY=${S3_GANGLIA_MASTER_DISCOVERY}/
+ fi
+
+ if [ -z "$S3_GANGLIA_BOOTSTRAP_SUCCESS" ]; then
+ terminate "Ganglia master success URL doesn't specified"
+ fi
+
+ if [[ "$S3_GANGLIA_BOOTSTRAP_SUCCESS" != */ ]]; then
+ S3_GANGLIA_BOOTSTRAP_SUCCESS=${S3_GANGLIA_BOOTSTRAP_SUCCESS}/
+ fi
+
+ if [ -z "$S3_GANGLIA_BOOTSTRAP_FAILURE" ]; then
+ terminate "Ganglia master failure URL doesn't specified"
+ fi
+
+ if [[ "$S3_GANGLIA_BOOTSTRAP_FAILURE" != */ ]]; then
+ S3_GANGLIA_BOOTSTRAP_FAILURE=${S3_GANGLIA_BOOTSTRAP_FAILURE}/
+ fi
+}
+
+# Prints EC2 instance info
+printInstanceInfo()
+{
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "[INFO] Cassandra download URL: $CASSANDRA_DOWNLOAD_URL"
+ echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+ echo "[INFO] Ganglia Core download URL: $GANGLIA_CORE_DOWNLOAD_URL"
+ echo "[INFO] Ganglia Web download URL: $GANGLIA_WEB_DOWNLOAD_URL"
+ echo "[INFO] RRD download URL: $RRD_DOWNLOAD_URL"
+ echo "[INFO] Logs URL: $S3_CASSANDRA_LOGS"
+ echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER"
+ echo "[INFO] Cassandra nodes discovery URL: $S3_CASSANDRA_NODES_DISCOVERY"
+ echo "[INFO] Ganglia master discovery URL: $S3_GANGLIA_MASTER_DISCOVERY"
+ echo "[INFO] Cassandra first node lock URL: $S3_CASSANDRA_FIRST_NODE_LOCK"
+ echo "[INFO] Cassandra nodes join lock URL: $S3_CASSANDRA_NODES_JOIN_LOCK"
+ echo "[INFO] Cassandra success URL: $S3_CASSANDRA_BOOTSTRAP_SUCCESS"
+ echo "[INFO] Cassandra failure URL: $S3_CASSANDRA_BOOTSTRAP_FAILURE"
+ fi
+
+ if [ "$NODE_TYPE" == "ignite" ]; then
+ echo "[INFO] Ignite download URL: $IGNITE_DOWNLOAD_URL"
+ echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+ echo "[INFO] Ganglia Core download URL: $GANGLIA_CORE_DOWNLOAD_URL"
+ echo "[INFO] Ganglia Web download URL: $GANGLIA_WEB_DOWNLOAD_URL"
+ echo "[INFO] RRD download URL: $RRD_DOWNLOAD_URL"
+ echo "[INFO] Logs URL: $S3_IGNITE_LOGS"
+ echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER"
+ echo "[INFO] Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY"
+ echo "[INFO] Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY"
+ echo "[INFO] Ganglia master discovery URL: $S3_GANGLIA_MASTER_DISCOVERY"
+ echo "[INFO] Ignite first node lock URL: $S3_IGNITE_FIRST_NODE_LOCK"
+ echo "[INFO] Ignite nodes join lock URL: $S3_IGNITE_NODES_JOIN_LOCK"
+ echo "[INFO] Ignite success URL: $S3_IGNITE_BOOTSTRAP_SUCCESS"
+ echo "[INFO] Ignite failure URL: $S3_IGNITE_BOOTSTRAP_FAILURE"
+ fi
+
+ if [ "$NODE_TYPE" == "test" ]; then
+ echo "[INFO] Tests type: $TESTS_TYPE"
+ echo "[INFO] Test nodes count: $TEST_NODES_COUNT"
+ echo "[INFO] Ignite nodes count: $IGNITE_NODES_COUNT"
+ echo "[INFO] Cassandra nodes count: $CASSANDRA_NODES_COUNT"
+ echo "[INFO] Tests summary URL: $S3_TESTS_SUMMARY"
+ echo "[INFO] ----------------------------------------------------"
+ echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+ echo "[INFO] Ganglia Core download URL: $GANGLIA_CORE_DOWNLOAD_URL"
+ echo "[INFO] Ganglia Web download URL: $GANGLIA_WEB_DOWNLOAD_URL"
+ echo "[INFO] RRD download URL: $RRD_DOWNLOAD_URL"
+ echo "[INFO] Logs URL: $S3_TESTS_LOGS"
+ echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER"
+ echo "[INFO] Test node discovery URL: $S3_TESTS_NODES_DISCOVERY"
+ echo "[INFO] Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY"
+ echo "[INFO] Cassandra node discovery URL: $S3_CASSANDRA_NODES_DISCOVERY"
+ echo "[INFO] Ganglia master discovery URL: $S3_GANGLIA_MASTER_DISCOVERY"
+ echo "[INFO] Tests trigger URL: $S3_TESTS_TRIGGER"
+ echo "[INFO] Tests idle URL: $S3_TESTS_IDLE"
+ echo "[INFO] Tests preparing URL: $S3_TESTS_PREPARING"
+ echo "[INFO] Tests waiting URL: $S3_TESTS_WAITING"
+ echo "[INFO] Tests running URL: $S3_TESTS_RUNNING"
+ echo "[INFO] Tests success URL: $S3_TESTS_SUCCESS"
+ echo "[INFO] Tests failure URL: $S3_TESTS_FAILURE"
+ echo "[INFO] Ignite success URL: $S3_IGNITE_BOOTSTRAP_SUCCESS"
+ echo "[INFO] Ignite failure URL: $S3_IGNITE_BOOTSTRAP_FAILURE"
+ echo "[INFO] Cassandra success URL: $S3_CASSANDRA_BOOTSTRAP_SUCCESS"
+ echo "[INFO] Cassandra failure URL: $S3_CASSANDRA_BOOTSTRAP_FAILURE"
+ fi
+
+ if [ "$NODE_TYPE" == "ganglia" ]; then
+ echo "[INFO] Ganglia Core download URL: $GANGLIA_CORE_DOWNLOAD_URL"
+ echo "[INFO] Ganglia Web download URL: $GANGLIA_WEB_DOWNLOAD_URL"
+ echo "[INFO] RRD download URL: $RRD_DOWNLOAD_URL"
+ echo "[INFO] Tests package download URL: $TESTS_PACKAGE_DONLOAD_URL"
+ echo "[INFO] Logs URL: $S3_GANGLIA_LOGS"
+ echo "[INFO] Logs trigger URL: $S3_LOGS_TRIGGER"
+ echo "[INFO] Ganglia master discovery URL: $S3_GANGLIA_MASTER_DISCOVERY"
+ echo "[INFO] Ganglia success URL: $S3_GANGLIA_BOOTSTRAP_SUCCESS"
+ echo "[INFO] Ganglia failure URL: $S3_GANGLIA_BOOTSTRAP_FAILURE"
+ fi
+}
+
+# Clone git repository
+gitClone()
+{
+ echo "[INFO] Cloning git repository $1 to $2"
+
+ rm -Rf $2
+
+ for i in 0 9;
+ do
+ git clone $1 $2
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] Git repository $1 was successfully cloned to $2"
+ return 0
+ fi
+
+ echo "[WARN] Failed to clone git repository $1 from $i attempt, sleeping extra 5sec"
+ rm -Rf $2
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to clone git repository $1 are failed"
+}
+
+# Applies specified tag to EC2 instance
+createTag()
+{
+ if [ -z "$EC2_INSTANCE_REGION" ]; then
+ EC2_AVAIL_ZONE=`curl -s http://169.254.169.254/latest/meta-data/placement/availability-zone`
+ EC2_INSTANCE_REGION="`echo \"$EC2_AVAIL_ZONE\" | sed -e 's:\([0-9][0-9]*\)[a-z]*\$:\\1:'`"
+ export EC2_INSTANCE_REGION
+ echo "[INFO] EC2 instance region: $EC2_INSTANCE_REGION"
+ fi
+
+ for i in 0 9;
+ do
+ aws ec2 create-tags --resources $1 --tags Key=$2,Value=$3 --region $EC2_INSTANCE_REGION
+ if [ $? -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[WARN] $i attempt to tag EC2 instance $1 with $2=$3 is failed, sleeping extra 5sec"
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to tag EC2 instance $1 with $2=$3 are failed"
+}
+
+# Applies 'owner', 'project' and 'Name' tags to EC2 instance
+tagInstance()
+{
+ export EC2_HOME=/opt/aws/apitools/ec2
+ export JAVA_HOME=/opt/java
+ export PATH=$JAVA_HOME/bin:$EC2_HOME/bin:$PATH
+
+ INSTANCE_ID=$(curl http://169.254.169.254/latest/meta-data/instance-id)
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to get instance metadata to tag it"
+ exit 1
+ fi
+
+ INSTANCE_NAME=
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ INSTANCE_NAME=$EC2_CASSANDRA_TAG
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ INSTANCE_NAME=$EC2_IGNITE_TAG
+ elif [ "$NODE_TYPE" == "test" ]; then
+ INSTANCE_NAME=$EC2_TEST_TAG
+ elif [ "$NODE_TYPE" == "ganglia" ]; then
+ INSTANCE_NAME=$EC2_GANGLIA_TAG
+ fi
+
+ if [ -n "$INSTANCE_NAME" ]; then
+ createTag "$INSTANCE_ID" "Name" "${INSTANCE_NAME}"
+ fi
+
+ if [ -n "$EC2_OWNER_TAG" ]; then
+ createTag "$INSTANCE_ID" "owner" "${EC2_OWNER_TAG}"
+ fi
+
+ if [ -n "$EC2_PROJECT_TAG" ]; then
+ createTag "$INSTANCE_ID" "project" "${EC2_PROJECT_TAG}"
+ fi
+}
+
+# Sets NODE_TYPE env variable
+setNodeType()
+{
+ if [ -n "$1" ]; then
+ NEW_NODE_TYPE=$NODE_TYPE
+ NODE_TYPE=$1
+ else
+ NEW_NODE_TYPE=
+ fi
+}
+
+# Reverts NODE_TYPE env variable to previous value
+revertNodeType()
+{
+ if [ -n "$NEW_NODE_TYPE" ]; then
+ NODE_TYPE=$NEW_NODE_TYPE
+ NEW_NODE_TYPE=
+ fi
+}
+
+# Returns logs folder for the node (Cassandra, Ignite, Tests)
+getLocalLogsFolder()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "/opt/cassandra/logs"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "/opt/ignite/work/log"
+ elif [ "$NODE_TYPE" == "test" ]; then
+ echo "/opt/ignite-cassandra-tests/logs"
+ elif [ "$NODE_TYPE" == "ganglia" ]; then
+ echo ""
+ fi
+
+ revertNodeType
+}
+
+# Returns S3 URL to discover this node
+getDiscoveryUrl()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "$S3_CASSANDRA_NODES_DISCOVERY"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "$S3_IGNITE_NODES_DISCOVERY"
+ elif [ "$NODE_TYPE" == "test" ]; then
+ echo "$S3_TESTS_NODES_DISCOVERY"
+ elif [ "$NODE_TYPE" == "ganglia" ]; then
+ echo "$S3_GANGLIA_MASTER_DISCOVERY"
+ fi
+
+ revertNodeType
+}
+
+# Returns S3 URL used as a join lock, used by nodes to join cluster sequentially
+getJoinLockUrl()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "$S3_CASSANDRA_NODES_JOIN_LOCK"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "$S3_IGNITE_NODES_JOIN_LOCK"
+ fi
+
+ revertNodeType
+}
+
+# Returns S3 URL used to select first node for the cluster. The first node is responsible
+# for doing all routine work (clean S3 logs/test results from previous execution) on cluster startup
+getFirstNodeLockUrl()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "$S3_CASSANDRA_FIRST_NODE_LOCK"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "$S3_IGNITE_FIRST_NODE_LOCK"
+ elif [ "$NODE_TYPE" == "test" ]; then
+ echo "$S3_TESTS_FIRST_NODE_LOCK"
+ fi
+
+ revertNodeType
+}
+
+# Returns S3 success URL for the node - folder created in S3 in case node successfully started and containing node logs
+getSucessUrl()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "$S3_CASSANDRA_BOOTSTRAP_SUCCESS"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "$S3_IGNITE_BOOTSTRAP_SUCCESS"
+ elif [ "$NODE_TYPE" == "test" ]; then
+ echo "$S3_TESTS_SUCCESS"
+ elif [ "$NODE_TYPE" == "ganglia" ]; then
+ echo "$S3_GANGLIA_BOOTSTRAP_SUCCESS"
+ fi
+
+ revertNodeType
+}
+
+# Returns S3 failure URL for the node - folder created in S3 in case node failed to start and containing node logs
+getFailureUrl()
+{
+ setNodeType $1
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ echo "$S3_CASSANDRA_BOOTSTRAP_FAILURE"
+ elif [ "$NODE_TYPE" == "ignite" ]; then
+ echo "$S3_IGNITE_BOOTSTRAP_FAILURE"
+ elif [ "$NODE_TYPE" == "test" ]; then
+ echo "$S3_TESTS_FAILURE"
+ elif [ "$NODE_TYPE" == "ganglia" ]; then
+ echo "$S3_GANGLIA_BOOTSTRAP_FAILURE"
+ fi
+
+ revertNodeType
+}
+
+# Terminates script execution, unregisters node and removes all the locks (join lock, first node lock) created by it
+terminate()
+{
+ SUCCESS_URL=$(getSucessUrl)
+ FAILURE_URL=$(getFailureUrl)
+
+ if [ -n "$SUCCESS_URL" ] && [[ "$SUCCESS_URL" != */ ]]; then
+ SUCCESS_URL=${SUCCESS_URL}/
+ fi
+
+ if [ -n "$FAILURE_URL" ] && [[ "$FAILURE_URL" != */ ]]; then
+ FAILURE_URL=${FAILURE_URL}/
+ fi
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ msg=$HOST_NAME
+
+ if [ -n "$1" ]; then
+ echo "[ERROR] $1"
+ echo "[ERROR]-----------------------------------------------------"
+ echo "[ERROR] Failed to start $NODE_TYPE node"
+ echo "[ERROR]-----------------------------------------------------"
+ msg=$1
+ reportFolder=${FAILURE_URL}${HOST_NAME}
+ reportFile=$reportFolder/__error__
+ else
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] $NODE_TYPE node successfully started"
+ echo "[INFO]-----------------------------------------------------"
+ reportFolder=${SUCCESS_URL}${HOST_NAME}
+ reportFile=$reportFolder/__success__
+ fi
+
+ echo $msg > /opt/ignite-cassandra-tests/bootstrap/start_result
+
+ aws s3 rm --recursive $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop report folder: $reportFolder"
+ fi
+
+ localLogs=$(getLocalLogsFolder)
+
+ if [ -d "$localLogs" ]; then
+ aws s3 sync --sse AES256 $localLogs $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to export $NODE_TYPE logs to: $reportFolder"
+ fi
+ fi
+
+ aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/bootstrap/start_result $reportFile
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to export node start result to: $reportFile"
+ fi
+
+ rm -f /opt/ignite-cassandra-tests/bootstrap/start_result /opt/ignite-cassandra-tests/bootstrap/join-lock /opt/ignite-cassandra-tests/bootstrap/first-node-lock
+
+ removeClusterJoinLock
+
+ if [ "$NODE_TYPE" == "test" ]; then
+ aws s3 rm ${S3_TESTS_RUNNING}${HOST_NAME}
+ aws s3 rm ${S3_TESTS_WAITING}${HOST_NAME}
+ aws s3 rm ${S3_TESTS_IDLE}${HOST_NAME}
+ aws s3 rm ${S3_TESTS_PREPARING}${HOST_NAME}
+ unregisterNode
+ fi
+
+ if [ -n "$1" ]; then
+ removeFirstNodeLock
+ unregisterNode
+ exit 1
+ fi
+
+ exit 0
+}
+
+# Registers node by creating a file having node hostname inside specific folder in S3
+registerNode()
+{
+ DISCOVERY_URL=$(getDiscoveryUrl)
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Registering $NODE_TYPE node: ${DISCOVERY_URL}${HOST_NAME}"
+
+ aws s3 cp --sse AES256 /etc/hosts ${DISCOVERY_URL}${HOST_NAME}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to register $NODE_TYPE node info in: ${DISCOVERY_URL}${HOST_NAME}"
+ fi
+
+ echo "[INFO] $NODE_TYPE node successfully registered"
+}
+
+# Unregisters node by removing a file having node hostname inside specific folder in S3
+unregisterNode()
+{
+ DISCOVERY_URL=$(getDiscoveryUrl)
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Removing $NODE_TYPE node registration from: ${DISCOVERY_URL}${HOST_NAME}"
+
+ exists=$(aws s3 ls ${DISCOVERY_URL}${HOST_NAME})
+
+ if [ -n "$exists" ]; then
+ aws s3 rm ${DISCOVERY_URL}${HOST_NAME}
+
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to remove $NODE_TYPE node registration"
+ else
+ echo "[INFO] $NODE_TYPE node registration removed"
+ fi
+ else
+ echo "[INFO] Node registration actually haven't been previously created"
+ fi
+}
+
+# Cleans up all nodes metadata for particular cluster (Cassandra, Ignite, Tests). Performed only by the node acquired
+# first node lock.
+cleanupMetadata()
+{
+ DISCOVERY_URL=$(getDiscoveryUrl)
+ JOIN_LOCK_URL=$(getJoinLockUrl)
+ SUCCESS_URL=$(getSucessUrl)
+ FAILURE_URL=$(getFailureUrl)
+
+ echo "[INFO] Running metadata cleanup"
+
+ aws s3 rm $JOIN_LOCK_URL
+ aws s3 rm --recursive $DISCOVERY_URL
+ aws s3 rm --recursive $SUCCESS_URL
+ aws s3 rm --recursive $FAILURE_URL
+
+ echo "[INFO] Metadata cleanup completed"
+}
+
+# Tries to get first node lock for the node. Only one (first) node can have such lock and it will be responsible for
+# cleanup process when starting cluster
+tryToGetFirstNodeLock()
+{
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ return 0
+ fi
+
+ FIRST_NODE_LOCK_URL=$(getFirstNodeLockUrl)
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Trying to get first node lock: $FIRST_NODE_LOCK_URL"
+
+ checkFirstNodeLockExist $FIRST_NODE_LOCK_URL
+ if [ $? -ne 0 ]; then
+ return 1
+ fi
+
+ echo "$HOST_NAME" > /opt/ignite-cassandra-tests/bootstrap/first-node-lock
+
+ createFirstNodeLock $FIRST_NODE_LOCK_URL
+
+ sleep 5s
+
+ rm -Rf /opt/ignite-cassandra-tests/bootstrap/first-node-lock
+
+ aws s3 cp $FIRST_NODE_LOCK_URL /opt/ignite-cassandra-tests/bootstrap/first-node-lock
+ if [ $? -ne 0 ]; then
+ echo "[WARN] Failed to check just created first node lock"
+ return 1
+ fi
+
+ first_host=$(cat /opt/ignite-cassandra-tests/bootstrap/first-node-lock)
+
+ rm -f /opt/ignite-cassandra-tests/bootstrap/first-node-lock
+
+ if [ "$first_host" != "$HOST_NAME" ]; then
+ echo "[INFO] Node $first_host has discarded previously created first node lock"
+ return 1
+ fi
+
+ echo "[INFO] Congratulations, got first node lock"
+
+ FIRST_NODE_LOCK="true"
+
+ return 0
+}
+
+# Checks if first node lock already exists in S3
+checkFirstNodeLockExist()
+{
+ echo "[INFO] Checking for the first node lock: $1"
+
+ lockExists=$(aws s3 ls $1)
+ if [ -n "$lockExists" ]; then
+ echo "[INFO] First node lock already exists"
+ return 1
+ fi
+
+ echo "[INFO] First node lock doesn't exist"
+
+ return 0
+}
+
+# Creates first node lock in S3
+createFirstNodeLock()
+{
+ aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/bootstrap/first-node-lock $1
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create first node lock: $1"
+ fi
+
+ echo "[INFO] Created first node lock: $1"
+}
+
+# Removes first node lock from S3
+removeFirstNodeLock()
+{
+ if [ "$FIRST_NODE_LOCK" != "true" ]; then
+ return 0
+ fi
+
+ FIRST_NODE_LOCK_URL=$(getFirstNodeLockUrl)
+
+ echo "[INFO] Removing first node lock: $FIRST_NODE_LOCK_URL"
+
+ aws s3 rm $FIRST_NODE_LOCK_URL
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to remove first node lock: $FIRST_NODE_LOCK_URL"
+ fi
+
+ echo "[INFO] Removed first node lock: $FIRST_NODE_LOCK_URL"
+
+ FIRST_NODE_LOCK="false"
+}
+
+# Tries to get cluster join lock. Nodes use this lock to join a cluster sequentially.
+tryToGetClusterJoinLock()
+{
+ if [ "$JOIN_LOCK" == "true" ]; then
+ return 0
+ fi
+
+ JOIN_LOCK_URL=$(getJoinLockUrl)
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Trying to get cluster join lock"
+
+ checkClusterJoinLockExist $JOIN_LOCK_URL
+ if [ $? -ne 0 ]; then
+ return 1
+ fi
+
+ echo "$HOST_NAME" > /opt/ignite-cassandra-tests/bootstrap/join-lock
+
+ createClusterJoinLock $JOIN_LOCK_URL
+
+ sleep 5s
+
+ rm -Rf /opt/ignite-cassandra-tests/bootstrap/join-lock
+
+ aws s3 cp $JOIN_LOCK_URL /opt/ignite-cassandra-tests/bootstrap/join-lock
+ if [ $? -ne 0 ]; then
+ echo "[WARN] Failed to check just created cluster join lock"
+ return 1
+ fi
+
+ join_host=$(cat /opt/ignite-cassandra-tests/bootstrap/join-lock)
+
+ if [ "$join_host" != "$HOST_NAME" ]; then
+ echo "[INFO] Node $first_host has discarded previously created cluster join lock"
+ return 1
+ fi
+
+ echo "[INFO] Congratulations, got cluster join lock"
+
+ JOIN_LOCK="true"
+
+ return 0
+}
+
+# Checks if join lock already exists in S3
+checkClusterJoinLockExist()
+{
+ echo "[INFO] Checking for the cluster join lock: $1"
+
+ lockExists=$(aws s3 ls $1)
+ if [ -n "$lockExists" ]; then
+ echo "[INFO] Cluster join lock already exists"
+ return 1
+ fi
+
+ if [ "$NODE_TYPE" == "cassandra" ]; then
+ status=$(/opt/cassandra/bin/nodetool -h $CASSANDRA_SEED status)
+ leaving=$(echo $status | grep UL)
+ moving=$(echo $status | grep UM)
+ joining=$(echo $status | grep UJ)
+
+ if [ -n "$leaving" ] || [ -n "$moving" ] || [ -n "$joining" ]; then
+ echo "[INFO] Cluster join lock doesn't exist in S3, but some node still trying to join Cassandra cluster"
+ return 1
+ fi
+ fi
+
+ echo "[INFO] Cluster join lock doesn't exist"
+}
+
+# Creates join lock in S3
+createClusterJoinLock()
+{
+ aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/bootstrap/join-lock $1
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create cluster join lock: $1"
+ fi
+
+ echo "[INFO] Created cluster join lock: $1"
+}
+
+# Removes join lock
+removeClusterJoinLock()
+{
+ if [ "$JOIN_LOCK" != "true" ]; then
+ return 0
+ fi
+
+ JOIN_LOCK_URL=$(getJoinLockUrl)
+
+ echo "[INFO] Removing cluster join lock: $JOIN_LOCK_URL"
+
+ aws s3 rm $JOIN_LOCK_URL
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to remove cluster join lock: $JOIN_LOCK_URL"
+ fi
+
+ JOIN_LOCK="false"
+
+ echo "[INFO] Removed cluster join lock: $JOIN_LOCK_URL"
+}
+
+# Waits for the node to join cluster, periodically trying to acquire cluster join lock and exiting only when node
+# successfully acquired the lock. Such mechanism used by nodes to join cluster sequentially (limitation of Cassandra).
+waitToJoinCluster()
+{
+ echo "[INFO] Waiting to join $NODE_TYPE cluster"
+
+ while true; do
+ tryToGetClusterJoinLock
+
+ if [ $? -ne 0 ]; then
+ echo "[INFO] Another node is trying to join cluster. Waiting for extra 30sec."
+ sleep 30s
+ else
+ echo "[INFO]-------------------------------------------------------------"
+ echo "[INFO] Congratulations, got lock to join $NODE_TYPE cluster"
+ echo "[INFO]-------------------------------------------------------------"
+ break
+ fi
+ done
+}
+
+# Wait for the cluster to register at least one node in S3, so that all other nodes will use already existing nodes
+# to send them info about them and join the cluster
+setupClusterSeeds()
+{
+ if [ "$1" != "cassandra" ] && [ "$1" != "ignite" ] && [ "$1" != "test" ]; then
+ terminate "Incorrect cluster type specified '$1' to setup seeds"
+ fi
+
+ DISCOVERY_URL=$(getDiscoveryUrl $1)
+
+ echo "[INFO] Setting up $1 seeds"
+
+ echo "[INFO] Looking for $1 seeds in: $DISCOVERY_URL"
+
+ startTime=$(date +%s)
+
+ while true; do
+ seeds=$(aws s3 ls $DISCOVERY_URL | grep -v PRE | sed -r "s/^.* //g")
+ if [ -n "$seeds" ]; then
+ seeds=($seeds)
+ length=${#seeds[@]}
+
+ if [ $length -lt 4 ]; then
+ seed1=${seeds[0]}
+ seed2=${seeds[1]}
+ seed3=${seeds[2]}
+ else
+ pos1=$(($RANDOM%$length))
+ pos2=$(($RANDOM%$length))
+ pos3=$(($RANDOM%$length))
+ seed1=${seeds[${pos1}]}
+ seed2=${seeds[${pos2}]}
+ seed3=${seeds[${pos3}]}
+ fi
+
+ CLUSTER_SEEDS=$seed1
+
+ if [ "$seed2" != "$seed1" ] && [ -n "$seed2" ]; then
+ CLUSTER_SEEDS="$CLUSTER_SEEDS $seed2"
+ fi
+
+ if [ "$seed3" != "$seed2" ] && [ "$seed3" != "$seed1" ] && [ -n "$seed3" ]; then
+ CLUSTER_SEEDS="$CLUSTER_SEEDS $seed3"
+ fi
+
+ echo "[INFO] Using $1 seeds: $CLUSTER_SEEDS"
+
+ return 0
+ fi
+
+ currentTime=$(date +%s)
+ duration=$(( $currentTime-$startTime ))
+ duration=$(( $duration/60 ))
+
+ if [ "$2" == "true" ]; then
+ if [ $duration -gt $SERVICE_STARTUP_TIME ]; then
+ terminate "${SERVICE_STARTUP_TIME}min timeout expired, but first $1 node is still not up and running"
+ fi
+ fi
+
+ echo "[INFO] Waiting for the first $1 node to start and publish its seed, time passed ${duration}min"
+
+ sleep 30s
+ done
+}
+
+# Wait until first cluster node registered in S3
+waitFirstClusterNodeRegistered()
+{
+ DISCOVERY_URL=$(getDiscoveryUrl)
+
+ echo "[INFO] Waiting for the first $NODE_TYPE node to register in: $DISCOVERY_URL"
+
+ startTime=$(date +%s)
+
+ while true; do
+ exists=$(aws s3 ls $DISCOVERY_URL)
+ if [ -n "$exists" ]; then
+ break
+ fi
+
+ if [ "$1" == "true" ]; then
+ currentTime=$(date +%s)
+ duration=$(( $currentTime-$startTime ))
+ duration=$(( $duration/60 ))
+
+ if [ $duration -gt $SERVICE_STARTUP_TIME ]; then
+ terminate "${SERVICE_STARTUP_TIME}min timeout expired, but first $type node is still not up and running"
+ fi
+ fi
+
+ echo "[INFO] Waiting extra 30sec"
+
+ sleep 30s
+ done
+
+ echo "[INFO] First $type node registered"
+}
+
+# Waits until all cluster nodes successfully bootstrapped. In case of Tests cluster also waits until all nodes
+# switch to waiting state
+waitAllClusterNodesReady()
+{
+ if [ "$1" == "cassandra" ]; then
+ NODES_COUNT=$CASSANDRA_NODES_COUNT
+ elif [ "$1" == "ignite" ]; then
+ NODES_COUNT=$IGNITE_NODES_COUNT
+ elif [ "$1" == "test" ]; then
+ NODES_COUNT=$TEST_NODES_COUNT
+ else
+ terminate "Incorrect cluster type specified '$1' to wait for all nodes up and running"
+ fi
+
+ SUCCESS_URL=$(getSucessUrl $1)
+
+ if [ $NODES_COUNT -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[INFO] Waiting for all $NODES_COUNT $1 nodes ready"
+
+ while true; do
+ if [ "$1" == "test" ]; then
+ count1=$(aws s3 ls $S3_TESTS_WAITING | wc -l)
+ count2=$(aws s3 ls $S3_TESTS_RUNNING | wc -l)
+ count=$(( $count1+$count2 ))
+ else
+ count=$(aws s3 ls $SUCCESS_URL | wc -l)
+ fi
+
+ if [ $count -ge $NODES_COUNT ]; then
+ break
+ fi
+
+ echo "[INFO] Waiting extra 30sec"
+
+ sleep 30s
+ done
+
+ sleep 30s
+
+ echo "[INFO] Congratulation, all $NODES_COUNT $1 nodes are ready"
+}
+
+# Wait untill all Tests cluster nodes completed their tests execution
+waitAllTestNodesCompletedTests()
+{
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "[INFO] Waiting for all $TEST_NODES_COUNT test nodes to complete their tests"
+
+ while true; do
+
+ count=$(aws s3 ls $S3_TESTS_RUNNING | grep -v $HOST_NAME | wc -l)
+
+ if [ $count -eq 0 ]; then
+ break
+ fi
+
+ echo "[INFO] Waiting extra 30sec"
+
+ sleep 30s
+ done
+
+ echo "[INFO] Congratulation, all $TEST_NODES_COUNT test nodes have completed their tests"
+}
+
+# Installs all required Ganglia packages
+installGangliaPackages()
+{
+ if [ "$1" == "master" ]; then
+ echo "[INFO] Installing Ganglia master required packages"
+ else
+ echo "[INFO] Installing Ganglia agent required packages"
+ fi
+
+ isAmazonLinux=$(cat "/etc/issue" | grep "Amazon Linux")
+
+ if [ -z "$isAmazonLinux" ]; then
+ setenforce 0
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to turn off SELinux"
+ fi
+
+ downloadPackage "$EPEL_DOWNLOAD_URL" "/opt/epel.rpm" "EPEL"
+
+ rpm -Uvh /opt/epel.rpm
+ if [ $? -ne 0 ]; then
+ terminate "Failed to setup EPEL repository"
+ fi
+
+ rm -f /opt/epel.rpm
+ fi
+
+ yum -y install apr-devel apr-util check-devel cairo-devel pango-devel pango \
+ libxml2-devel glib2-devel dbus-devel freetype-devel freetype \
+ libpng-devel libart_lgpl-devel fontconfig-devel gcc-c++ expat-devel \
+ python-devel libXrender-devel perl-devel perl-CPAN gettext git sysstat \
+ automake autoconf ltmain.sh pkg-config gperf libtool pcre-devel libconfuse-devel
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install all Ganglia required packages"
+ fi
+
+ if [ "$1" == "master" ]; then
+ yum -y install httpd php php-devel php-pear
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install all Ganglia required packages"
+ fi
+
+ if [ -z "$isAmazonLinux" ]; then
+ yum -y install liberation-sans-fonts
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install liberation-sans-fonts package"
+ fi
+ fi
+ fi
+
+ if [ -z "$isAmazonLinux" ]; then
+ downloadPackage "$GPERF_DOWNLOAD_URL" "/opt/gperf.tar.gz" "gperf"
+
+ tar -xvzf /opt/gperf.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar gperf tarball"
+ fi
+
+ rm -Rf /opt/gperf.tar.gz
+
+ unzipDir=$(ls /opt | grep "gperf")
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to update creation date to current for all files inside: /opt/$unzipDir"
+ fi
+
+ pushd /opt/$unzipDir
+
+ cat ./configure | sed -r "s/test \"\\\$2\" = conftest.file/test 1 = 1/g" > ./configure1
+ rm ./configure
+ mv ./configure1 ./configure
+ chmod a+x ./configure
+
+ ./configure
+ if [ $? -ne 0 ]; then
+ terminate "Failed to configure gperf"
+ fi
+
+ make
+ if [ $? -ne 0 ]; then
+ terminate "Failed to make gperf"
+ fi
+
+ make install
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install gperf"
+ fi
+
+ echo "[INFO] gperf tool successfully installed"
+
+ popd
+ fi
+
+ echo "[INFO] Installing rrdtool"
+
+ downloadPackage "$RRD_DOWNLOAD_URL" "/opt/rrdtool.tar.gz" "rrdtool"
+
+ tar -xvzf /opt/rrdtool.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar rrdtool tarball"
+ fi
+
+ rm -Rf /opt/rrdtool.tar.gz
+
+ unzipDir=$(ls /opt | grep "rrdtool")
+ if [ "$unzipDir" != "rrdtool" ]; then
+ mv /opt/$unzipDir /opt/rrdtool
+ fi
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to update creation date to current for all files inside: /opt/rrdtool"
+ fi
+
+ export PKG_CONFIG_PATH=/usr/lib/pkgconfig/
+
+ pushd /opt/rrdtool
+
+ cat ./configure | sed -r "s/test \"\\\$2\" = conftest.file/test 1 = 1/g" > ./configure1
+ rm ./configure
+ mv ./configure1 ./configure
+ chmod a+x ./configure
+
+ ./configure --prefix=/usr/local/rrdtool
+ if [ $? -ne 0 ]; then
+ terminate "Failed to configure rrdtool"
+ fi
+
+ make
+ if [ $? -ne 0 ]; then
+ terminate "Failed to make rrdtool"
+ fi
+
+ make install
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install rrdtool"
+ fi
+
+ ln -s /usr/local/rrdtool/bin/rrdtool /usr/bin/rrdtool
+ mkdir -p /var/lib/ganglia/rrds
+
+ chown -R nobody:nobody /usr/local/rrdtool /var/lib/ganglia/rrds /usr/bin/rrdtool
+
+ rm -Rf /opt/rrdtool
+
+ popd
+
+ echo "[INFO] rrdtool successfully installed"
+
+ echo "[INFO] Installig ganglia-core"
+
+ gitClone $GANGLIA_CORE_DOWNLOAD_URL /opt/monitor-core
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to update creation date to current for all files inside: /opt/monitor-core"
+ fi
+
+ pushd /opt/monitor-core
+
+ git checkout efe9b5e5712ea74c04e3b15a06eb21900e18db40
+
+ ./bootstrap
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to prepare ganglia-core for compilation"
+ fi
+
+ cat ./configure | sed -r "s/test \"\\\$2\" = conftest.file/test 1 = 1/g" > ./configure1
+ rm ./configure
+ mv ./configure1 ./configure
+ chmod a+x ./configure
+
+ ./configure --with-gmetad --with-librrd=/usr/local/rrdtool
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to configure ganglia-core"
+ fi
+
+ make
+ if [ $? -ne 0 ]; then
+ terminate "Failed to make ganglia-core"
+ fi
+
+ make install
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install ganglia-core"
+ fi
+
+ rm -Rf /opt/monitor-core
+
+ popd
+
+ echo "[INFO] ganglia-core successfully installed"
+
+ if [ "$1" != "master" ]; then
+ return 0
+ fi
+
+ echo "[INFO] Installing ganglia-web"
+
+ gitClone $GANGLIA_WEB_DOWNLOAD_URL /opt/web
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to update creation date to current for all files inside: /opt/web"
+ fi
+
+ cat /opt/web/Makefile | sed -r "s/GDESTDIR = \/usr\/share\/ganglia-webfrontend/GDESTDIR = \/opt\/ganglia-web/g" > /opt/web/Makefile1
+ cat /opt/web/Makefile1 | sed -r "s/GCONFDIR = \/etc\/ganglia-web/GCONFDIR = \/opt\/ganglia-web/g" > /opt/web/Makefile2
+ cat /opt/web/Makefile2 | sed -r "s/GWEB_STATEDIR = \/var\/lib\/ganglia-web/GWEB_STATEDIR = \/opt\/ganglia-web/g" > /opt/web/Makefile3
+ cat /opt/web/Makefile3 | sed -r "s/APACHE_USER = www-data/APACHE_USER = apache/g" > /opt/web/Makefile4
+
+ rm -f /opt/web/Makefile
+ cp /opt/web/Makefile4 /opt/web/Makefile
+ rm -f /opt/web/Makefile1 /opt/web/Makefile2 /opt/web/Makefile3 /opt/web/Makefile4
+
+ pushd /opt/web
+
+ git checkout f2b19c7cacfc8c51921be801b92f8ed0bd4901ae
+
+ make
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to make ganglia-web"
+ fi
+
+ make install
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install ganglia-web"
+ fi
+
+ rm -Rf /opt/web
+
+ popd
+
+ echo "" >> /etc/httpd/conf/httpd.conf
+ echo "Alias /ganglia /opt/ganglia-web" >> /etc/httpd/conf/httpd.conf
+ echo "" >> /etc/httpd/conf/httpd.conf
+ echo " AllowOverride All" >> /etc/httpd/conf/httpd.conf
+ echo " Order allow,deny" >> /etc/httpd/conf/httpd.conf
+
+ if [ -z "$isAmazonLinux" ]; then
+ echo " Require all granted" >> /etc/httpd/conf/httpd.conf
+ fi
+
+ echo " Allow from all" >> /etc/httpd/conf/httpd.conf
+ echo " Deny from none" >> /etc/httpd/conf/httpd.conf
+ echo " " >> /etc/httpd/conf/httpd.conf
+
+ echo "[INFO] ganglia-web successfully installed"
+}
+
+# Setup ntpd service
+setupNTP()
+{
+ echo "[INFO] Installing ntp package"
+
+ yum -y install ntp
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install ntp package"
+ fi
+
+ echo "[INFO] Starting ntpd service"
+
+ service ntpd restart
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to restart ntpd service"
+ fi
+}
+
+# Installs and run Ganglia agent ('gmond' daemon)
+bootstrapGangliaAgent()
+{
+ echo "[INFO]-----------------------------------------------------------------"
+ echo "[INFO] Bootstrapping Ganglia agent"
+ echo "[INFO]-----------------------------------------------------------------"
+
+ installGangliaPackages
+
+ echo "[INFO] Running ganglia agent daemon to discover Ganglia master"
+
+ /opt/ignite-cassandra-tests/bootstrap/aws/ganglia/agent-start.sh $1 $2 > /opt/ganglia-agent.log &
+
+ echo "[INFO] Ganglia daemon job id: $!"
+}
+
+# Partitioning, formatting to ext4 and mounting all unpartitioned drives.
+# As a result env array MOUNT_POINTS provides all newly created mount points.
+mountUnpartitionedDrives()
+{
+ MOUNT_POINTS=
+
+ echo "[INFO] Mounting unpartitioned drives"
+
+ lsblk -V &> /dev/null
+
+ if [ $? -ne 0 ]; then
+ echo "[WARN] lsblk utility doesn't exist"
+ echo "[INFO] Installing util-linux-ng package"
+
+ yum -y install util-linux-ng
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install util-linux-ng package"
+ fi
+ fi
+
+ parted -v &> /dev/null
+
+ if [ $? -ne 0 ]; then
+ echo "[WARN] parted utility doesn't exist"
+ echo "[INFO] Installing parted package"
+
+ yum -y install parted
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install parted package"
+ fi
+ fi
+
+ drives=$(lsblk -io KNAME,TYPE | grep disk | sed -r "s/disk//g" | xargs)
+
+ echo "[INFO] Found HDDs: $drives"
+
+ unpartDrives=
+ partDrives=$(lsblk -io KNAME,TYPE | grep part | sed -r "s/[0-9]*//g" | sed -r "s/part//g" | xargs)
+
+ drives=($drives)
+ count=${#drives[@]}
+ iter=1
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ drive=${drives[$i]}
+
+ if [ -z "$drive" ]; then
+ continue
+ fi
+
+ isPartitioned=$(echo $partDrives | grep "$drive")
+
+ if [ -n "$isPartitioned" ]; then
+ continue
+ fi
+
+ echo "[INFO] Creating partition for the drive: $drive"
+
+ parted -s -a opt /dev/$drive mklabel gpt mkpart primary 0% 100%
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create partition for the drive: $drive"
+ fi
+
+ partition=$(lsblk -io KNAME,TYPE | grep part | grep $drive | sed -r "s/part//g" | xargs)
+
+ echo "[INFO] Successfully created partition $partition for the drive: $drive"
+
+ echo "[INFO] Formatting partition /dev/$partition to ext4"
+
+ mkfs.ext4 -F -q /dev/$partition
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to format partition: /dev/$partition"
+ fi
+
+ echo "[INFO] Partition /dev/$partition was successfully formatted to ext4"
+
+ echo "[INFO] Mounting partition /dev/$partition to /storage$iter"
+
+ mkdir -p /storage$iter
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create mount point directory: /storage$iter"
+ fi
+
+ echo "/dev/$partition /storage$iter ext4 defaults 1 1" >> /etc/fstab
+
+ mount /storage$iter
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to mount /storage$iter mount point for partition /dev/$partition"
+ fi
+
+ echo "[INFO] Partition /dev/$partition was successfully mounted to /storage$iter"
+
+ if [ -n "$MOUNT_POINTS" ]; then
+ MOUNT_POINTS="$MOUNT_POINTS "
+ fi
+
+ MOUNT_POINTS="${MOUNT_POINTS}/storage${iter}"
+
+ iter=$(($iter+1))
+ done
+
+ if [ -z "$MOUNT_POINTS" ]; then
+ echo "[INFO] All drives already have partitions created"
+ fi
+
+ MOUNT_POINTS=($MOUNT_POINTS)
+}
+
+# Creates storage directories for Cassandra: data files, commit log, saved caches.
+# As a result CASSANDRA_DATA_DIR, CASSANDRA_COMMITLOG_DIR, CASSANDRA_CACHES_DIR will point to appropriate directories.
+createCassandraStorageLayout()
+{
+ CASSANDRA_DATA_DIR=
+ CASSANDRA_COMMITLOG_DIR=
+ CASSANDRA_CACHES_DIR=
+
+ mountUnpartitionedDrives
+
+ echo "[INFO] Creating Cassandra storage layout"
+
+ count=${#MOUNT_POINTS[@]}
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ mountPoint=${MOUNT_POINTS[$i]}
+
+ if [ -z "$CASSANDRA_DATA_DIR" ]; then
+ CASSANDRA_DATA_DIR=$mountPoint
+ elif [ -z "$CASSANDRA_COMMITLOG_DIR" ]; then
+ CASSANDRA_COMMITLOG_DIR=$mountPoint
+ elif [ -z "$CASSANDRA_CACHES_DIR" ]; then
+ CASSANDRA_CACHES_DIR=$mountPoint
+ else
+ CASSANDRA_DATA_DIR="$CASSANDRA_DATA_DIR $mountPoint"
+ fi
+ done
+
+ if [ -z "$CASSANDRA_DATA_DIR" ]; then
+ CASSANDRA_DATA_DIR="/storage/cassandra/data"
+ else
+ CASSANDRA_DATA_DIR="$CASSANDRA_DATA_DIR/cassandra_data"
+ fi
+
+ if [ -z "$CASSANDRA_COMMITLOG_DIR" ]; then
+ CASSANDRA_COMMITLOG_DIR="/storage/cassandra/commitlog"
+ else
+ CASSANDRA_COMMITLOG_DIR="$CASSANDRA_COMMITLOG_DIR/cassandra_commitlog"
+ fi
+
+ if [ -z "$CASSANDRA_CACHES_DIR" ]; then
+ CASSANDRA_CACHES_DIR="/storage/cassandra/saved_caches"
+ else
+ CASSANDRA_CACHES_DIR="$CASSANDRA_CACHES_DIR/cassandra_caches"
+ fi
+
+ echo "[INFO] Cassandra data dir: $CASSANDRA_DATA_DIR"
+ echo "[INFO] Cassandra commit log dir: $CASSANDRA_COMMITLOG_DIR"
+ echo "[INFO] Cassandra saved caches dir: $CASSANDRA_CACHES_DIR"
+
+ dirs=("$CASSANDRA_DATA_DIR $CASSANDRA_COMMITLOG_DIR $CASSANDRA_CACHES_DIR")
+
+ count=${#dirs[@]}
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ directory=${dirs[$i]}
+
+ mkdir -p $directory
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create directory: $directory"
+ fi
+
+ chown -R cassandra:cassandra $directory
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to assign cassandra:cassandra as an owner of directory $directory"
+ fi
+ done
+
+ DATA_DIR_SPEC="\n"
+
+ dirs=($CASSANDRA_DATA_DIR)
+
+ count=${#dirs[@]}
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ dataDir=${dirs[$i]}
+ DATA_DIR_SPEC="${DATA_DIR_SPEC} - ${dataDir}\n"
+ done
+
+ CASSANDRA_DATA_DIR=$(echo $DATA_DIR_SPEC | sed -r "s/\//\\\\\//g")
+ CASSANDRA_COMMITLOG_DIR=$(echo $CASSANDRA_COMMITLOG_DIR | sed -r "s/\//\\\\\//g")
+ CASSANDRA_CACHES_DIR=$(echo $CASSANDRA_CACHES_DIR | sed -r "s/\//\\\\\//g")
+}
+
+# Attaches environment configuration settings
+. $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/env.sh
+
+# Validates environment settings
+validate
+
+# Validates node type of EC2 instance
+if [ "$1" != "cassandra" ] && [ "$1" != "ignite" ] && [ "$1" != "test" ] && [ "$1" != "ganglia" ]; then
+ echo "[ERROR] Unsupported node type specified: $1"
+ exit 1
+fi
+
+# Sets node type of EC2 instance
+export NODE_TYPE=$1
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/env.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/env.sh
new file mode 100644
index 000000000..031c5c3a1
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/env.sh
@@ -0,0 +1,113 @@
+#!/bin/sh
+
+#
+# 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 file specifies environment specific settings to bootstrap required infrastructure for:
+# -----------------------------------------------------------------------------------------------
+#
+# 1) Cassandra cluster
+# 2) Ignite cluster
+# 3) Tests cluster
+# 4) Ganglia agents to be installed on each clusters machine
+# 5) Ganglia master to collect metrics from agent and show graphs on Ganglia Web dashboard
+#
+# -----------------------------------------------------------------------------------------------
+
+# EC2 tagging related settings
+export EC2_OWNER_TAG=ignite@apache.org
+export EC2_PROJECT_TAG=ignite
+export EC2_CASSANDRA_TAG=CASSANDRA
+export EC2_IGNITE_TAG=IGNITE
+export EC2_TEST_TAG=TEST
+export EC2_GANGLIA_TAG=GANGLIA
+
+# Tests summary settings
+export CASSANDRA_NODES_COUNT=3
+export IGNITE_NODES_COUNT=3
+export TEST_NODES_COUNT=2
+export TESTS_TYPE="ignite"
+
+# Time (in minutes) to wait for Cassandra/Ignite node up and running and register it in S3
+export SERVICE_STARTUP_TIME=10
+
+# Number of attempts to start Cassandra/Ignite daemon
+export SERVICE_START_ATTEMPTS=3
+
+# Root S3 folder
+export S3_ROOT=s3:///
+
+# S3 folder for downloads. You should put here ignite load tests jar archive
+# (you can also download here other required artifacts like Cassandra, Ignite and etc)
+export S3_DOWNLOADS=$S3_ROOT/test
+
+# S3 root system folders where to store all infrastructure info
+export S3_SYSTEM=$S3_ROOT/test1
+
+# S3 system folders to store cluster specific info
+export S3_CASSANDRA_SYSTEM=$S3_SYSTEM/cassandra
+export S3_IGNITE_SYSTEM=$S3_SYSTEM/ignite
+export S3_TESTS_SYSTEM=$S3_SYSTEM/tests
+export S3_GANGLIA_SYSTEM=$S3_SYSTEM/ganglia
+
+# Logs related settings
+export S3_LOGS_TRIGGER=$S3_SYSTEM/logs-trigger
+export S3_LOGS_ROOT=$S3_SYSTEM/logs
+export S3_CASSANDRA_LOGS=$S3_LOGS_ROOT/cassandra
+export S3_IGNITE_LOGS=$S3_LOGS_ROOT/ignite
+export S3_TESTS_LOGS=$S3_LOGS_ROOT/tests
+export S3_GANGLIA_LOGS=$S3_LOGS_ROOT/ganglia
+
+# Cassandra related settings
+export CASSANDRA_DOWNLOAD_URL=http://archive.apache.org/dist/cassandra/3.5/apache-cassandra-3.5-bin.tar.gz
+export S3_CASSANDRA_BOOTSTRAP_SUCCESS=$S3_CASSANDRA_SYSTEM/success
+export S3_CASSANDRA_BOOTSTRAP_FAILURE=$S3_CASSANDRA_SYSTEM/failure
+export S3_CASSANDRA_NODES_DISCOVERY=$S3_CASSANDRA_SYSTEM/discovery
+export S3_CASSANDRA_FIRST_NODE_LOCK=$S3_CASSANDRA_SYSTEM/first-node-lock
+export S3_CASSANDRA_NODES_JOIN_LOCK=$S3_CASSANDRA_SYSTEM/join-lock
+
+# Ignite related settings
+export IGNITE_DOWNLOAD_URL=$S3_DOWNLOADS/apache-ignite-fabric-1.8.0-SNAPSHOT-bin.zip
+export S3_IGNITE_BOOTSTRAP_SUCCESS=$S3_IGNITE_SYSTEM/success
+export S3_IGNITE_BOOTSTRAP_FAILURE=$S3_IGNITE_SYSTEM/failure
+export S3_IGNITE_NODES_DISCOVERY=$S3_IGNITE_SYSTEM/discovery
+export S3_IGNITE_FIRST_NODE_LOCK=$S3_IGNITE_SYSTEM/first-node-lock
+export S3_IGNITE_NODES_JOIN_LOCK=$S3_IGNITE_SYSTEM/i-join-lock
+
+# Tests related settings
+export TESTS_PACKAGE_DONLOAD_URL=$S3_DOWNLOADS/ignite-cassandra-tests-1.8.0-SNAPSHOT.zip
+export S3_TESTS_TRIGGER=$S3_SYSTEM/tests-trigger
+export S3_TESTS_NODES_DISCOVERY=$S3_TESTS_SYSTEM/discovery
+export S3_TESTS_SUCCESS=$S3_TESTS_SYSTEM/success
+export S3_TESTS_FAILURE=$S3_TESTS_SYSTEM/failure
+export S3_TESTS_IDLE=$S3_TESTS_SYSTEM/idle
+export S3_TESTS_PREPARING=$S3_TESTS_SYSTEM/preparing
+export S3_TESTS_WAITING=$S3_TESTS_SYSTEM/waiting
+export S3_TESTS_RUNNING=$S3_TESTS_SYSTEM/running
+export S3_TESTS_FIRST_NODE_LOCK=$S3_TESTS_SYSTEM/first-node-lock
+export S3_TESTS_SUMMARY=$S3_SYSTEM/t-summary.zip
+
+# Ganglia related settings
+export GANGLIA_CORE_DOWNLOAD_URL=https://github.com/ganglia/monitor-core.git
+export GANGLIA_WEB_DOWNLOAD_URL=https://github.com/ganglia/ganglia-web.git
+export RRD_DOWNLOAD_URL=http://oss.oetiker.ch/rrdtool/pub/rrdtool-1.3.1.tar.gz
+export GPERF_DOWNLOAD_URL=http://ftp.gnu.org/gnu/gperf/gperf-3.0.3.tar.gz
+export EPEL_DOWNLOAD_URL=https://dl.fedoraproject.org/pub/epel/epel-release-latest-7.noarch.rpm
+export S3_GANGLIA_BOOTSTRAP_SUCCESS=$S3_GANGLIA_SYSTEM/success
+export S3_GANGLIA_BOOTSTRAP_FAILURE=$S3_GANGLIA_SYSTEM/failure
+export S3_GANGLIA_MASTER_DISCOVERY=$S3_GANGLIA_SYSTEM/discovery
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/agent-start.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/agent-start.sh
new file mode 100644
index 000000000..8e49c1844
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/agent-start.sh
@@ -0,0 +1,75 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Script to start Ganglia agent on EC2 node (used by agent-bootstrap.sh)
+# -----------------------------------------------------------------------------------------------
+
+. /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "ganglia"
+
+echo "[INFO] Running Ganglia agent discovery daemon for '$1' cluster using $2 port"
+
+# Waiting for the Ganglia master node up and running
+waitFirstClusterNodeRegistered
+
+DISCOVERY_URL=$(getDiscoveryUrl)
+
+masterNode=$(aws s3 ls $DISCOVERY_URL | head -1)
+masterNode=($masterNode)
+masterNode=${masterNode[3]}
+masterNode=$(echo $masterNode | xargs)
+
+if [ $? -ne 0 ] || [ -z "$masterNode" ]; then
+ echo "[ERROR] Failed to get Ganglia master node from: $DISCOVERY_URL"
+fi
+
+echo "[INFO] Got Ganglia master node: $masterNode"
+
+echo "[INFO] Creating gmond config file"
+
+/usr/local/sbin/gmond --default_config > /opt/gmond-default.conf
+
+cat /opt/gmond-default.conf | sed -r "s/deaf = no/deaf = yes/g" | \
+sed -r "s/name = \"unspecified\"/name = \"$1\"/g" | \
+sed -r "s/#bind_hostname/bind_hostname/g" | \
+sed "0,/mcast_join = 239.2.11.71/s/mcast_join = 239.2.11.71/host = $masterNode/g" | \
+sed -r "s/mcast_join = 239.2.11.71//g" | sed -r "s/bind = 239.2.11.71//g" | \
+sed -r "s/port = 8649/port = $2/g" | sed -r "s/retry_bind = true//g" > /opt/gmond.conf
+
+echo "[INFO] Running gmond daemon to report to gmetad on $masterNode"
+
+/usr/local/sbin/gmond --conf=/opt/gmond.conf -p /opt/gmond.pid
+
+sleep 2s
+
+if [ ! -f "/opt/gmond.pid" ]; then
+ echo "[ERROR] Failed to start gmond daemon, pid file doesn't exist"
+ exit 1
+fi
+
+pid=$(cat /opt/gmond.pid)
+
+echo "[INFO] gmond daemon started, pid=$pid"
+
+exists=$(ps $pid | grep gmond)
+
+if [ -z "$exists" ]; then
+ echo "[ERROR] gmond daemon abnormally terminated"
+ exit 1
+fi
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/ganglia-bootstrap.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/ganglia-bootstrap.sh
new file mode 100644
index 000000000..15fa04455
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ganglia/ganglia-bootstrap.sh
@@ -0,0 +1,417 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Bootstrap script to spin up Ganglia master
+# -----------------------------------------------------------------------------------------------
+
+# URL to download AWS CLI tools
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+# URL to download JDK
+JDK_DOWNLOAD_URL=http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz
+
+# URL to download Ignite-Cassandra tests package - you should previously package and upload it to this place
+TESTS_PACKAGE_DONLOAD_URL=s3:////ignite-cassandra-tests-.zip
+
+# Terminates script execution and upload logs to S3
+terminate()
+{
+ SUCCESS_URL=$S3_GANGLIA_BOOTSTRAP_SUCCESS
+ FAILURE_URL=$S3_GANGLIA_BOOTSTRAP_FAILURE
+
+ if [ -n "$SUCCESS_URL" ] && [[ "$SUCCESS_URL" != */ ]]; then
+ SUCCESS_URL=${SUCCESS_URL}/
+ fi
+
+ if [ -n "$FAILURE_URL" ] && [[ "$FAILURE_URL" != */ ]]; then
+ FAILURE_URL=${FAILURE_URL}/
+ fi
+
+ host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ msg=$host_name
+
+ if [ -n "$1" ]; then
+ echo "[ERROR] $1"
+ echo "[ERROR]-----------------------------------------------------"
+ echo "[ERROR] Ganglia master node bootstrap failed"
+ echo "[ERROR]-----------------------------------------------------"
+ msg=$1
+
+ if [ -z "$FAILURE_URL" ]; then
+ exit 1
+ fi
+
+ reportFolder=${FAILURE_URL}${host_name}
+ reportFile=$reportFolder/__error__
+ else
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Ganglia master node bootstrap successfully completed"
+ echo "[INFO]-----------------------------------------------------"
+
+ if [ -z "$SUCCESS_URL" ]; then
+ exit 0
+ fi
+
+ reportFolder=${SUCCESS_URL}${host_name}
+ reportFile=$reportFolder/__success__
+ fi
+
+ echo $msg > /opt/bootstrap-result
+
+ aws s3 rm --recursive $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop report folder: $reportFolder"
+ fi
+
+ aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+ fi
+
+ rm -f /opt/bootstrap-result
+
+ if [ -n "$1" ]; then
+ exit 1
+ fi
+
+ exit 0
+}
+
+# Downloads specified package
+downloadPackage()
+{
+ echo "[INFO] Downloading $3 package from $1 into $2"
+
+ for i in 0 9;
+ do
+ if [[ "$1" == s3* ]]; then
+ aws s3 cp $1 $2
+ code=$?
+ else
+ curl "$1" -o "$2"
+ code=$?
+ fi
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] $3 package successfully downloaded from $1 into $2"
+ return 0
+ fi
+
+ echo "[WARN] Failed to download $3 package from $i attempt, sleeping extra 5sec"
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to download $3 package from $1 are failed"
+}
+
+# Downloads and setup JDK
+setupJava()
+{
+ rm -Rf /opt/java /opt/jdk.tar.gz
+
+ echo "[INFO] Downloading 'jdk'"
+ wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "$JDK_DOWNLOAD_URL" -O /opt/jdk.tar.gz
+ if [ $? -ne 0 ]; then
+ terminate "Failed to download 'jdk'"
+ fi
+
+ echo "[INFO] Untaring 'jdk'"
+ tar -xvzf /opt/jdk.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar 'jdk'"
+ fi
+
+ rm -Rf /opt/jdk.tar.gz
+
+ unzipDir=$(ls /opt | grep "jdk")
+ if [ "$unzipDir" != "java" ]; then
+ mv /opt/$unzipDir /opt/java
+ fi
+}
+
+# Downloads and setup AWS CLI
+setupAWSCLI()
+{
+ echo "[INFO] Installing 'awscli'"
+ pip install --upgrade awscli
+ if [ $? -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[ERROR] Failed to install 'awscli' using pip"
+ echo "[INFO] Trying to install awscli using zip archive"
+ echo "[INFO] Downloading awscli zip"
+
+ downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+ echo "[INFO] Unzipping awscli zip"
+ unzip /opt/awscli-bundle.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip awscli zip"
+ fi
+
+ rm -Rf /opt/awscli-bundle.zip
+
+ echo "[INFO] Installing awscli"
+ /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install awscli"
+ fi
+
+ echo "[INFO] Successfully installed awscli from zip archive"
+}
+
+# Setup all the pre-requisites (packages, settings and etc.)
+setupPreRequisites()
+{
+ echo "[INFO] Installing 'wget' package"
+ yum -y install wget
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'wget' package"
+ fi
+
+ echo "[INFO] Installing 'net-tools' package"
+ yum -y install net-tools
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'net-tools' package"
+ fi
+
+ echo "[INFO] Installing 'python' package"
+ yum -y install python
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'python' package"
+ fi
+
+ echo "[INFO] Installing 'unzip' package"
+ yum -y install unzip
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'unzip' package"
+ fi
+
+ downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+ echo "[INFO] Installing 'pip'"
+ python /opt/get-pip.py
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'pip'"
+ fi
+}
+
+# Downloads and setup tests package
+setupTestsPackage()
+{
+ downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/ignite-cassandra-tests.zip" "Tests"
+
+ rm -Rf /opt/ignite-cassandra-tests
+
+ unzip /opt/ignite-cassandra-tests.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip tests package"
+ fi
+
+ rm -f /opt/ignite-cassandra-tests.zip
+
+ unzipDir=$(ls /opt | grep "ignite-cassandra")
+ if [ "$unzipDir" != "ignite-cassandra-tests" ]; then
+ mv /opt/$unzipDir /opt/ignite-cassandra-tests
+ fi
+
+ find /opt/ignite-cassandra-tests -type f -name "*.sh" -exec chmod ug+x {} \;
+
+ . /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "ganglia"
+
+ setupNTP
+
+ echo "[INFO] Starting logs collector daemon"
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ /opt/ignite-cassandra-tests/bootstrap/aws/logs-collector.sh "$S3_LOGS_TRIGGER" "$S3_GANGLIA_LOGS/$HOST_NAME" "/var/log/httpd" > /opt/logs-collector.log &
+
+ echo "[INFO] Logs collector daemon started: $!"
+
+ echo "----------------------------------------------------------------------------------------"
+ printInstanceInfo
+ echo "----------------------------------------------------------------------------------------"
+ tagInstance
+}
+
+# Creates config file for 'gmond' damon working in receiver mode
+createGmondReceiverConfig()
+{
+ /usr/local/sbin/gmond --default_config > /opt/gmond-default.conf
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create gmond default config in: /opt/gmond-default.txt"
+ fi
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ cat /opt/gmond-default.conf | sed -r "s/mute = no/mute = yes/g" | \
+ sed -r "s/name = \"unspecified\"/name = \"$1\"/g" | \
+ sed -r "s/#bind_hostname/bind_hostname/g" | \
+ sed "0,/mcast_join = 239.2.11.71/s/mcast_join = 239.2.11.71/host = $HOST_NAME/g" | \
+ sed -r "s/mcast_join = 239.2.11.71//g" | sed -r "s/bind = 239.2.11.71//g" | \
+ sed -r "s/port = 8649/port = $2/g" | sed -r "s/retry_bind = true//g" > /opt/gmond-${1}.conf
+
+ chmod a+r /opt/gmond-${1}.conf
+
+ rm -f /opt/gmond-default.conf
+}
+
+# Creates config file for 'gmond' damon working in sender-receiver mode
+createGmondSenderReceiverConfig()
+{
+ /usr/local/sbin/gmond --default_config > /opt/gmond-default.conf
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create gmond default config in: /opt/gmond-default.txt"
+ fi
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ cat /opt/gmond-default.conf | sed -r "s/name = \"unspecified\"/name = \"$1\"/g" | \
+ sed -r "s/#bind_hostname/bind_hostname/g" | \
+ sed "0,/mcast_join = 239.2.11.71/s/mcast_join = 239.2.11.71/host = $HOST_NAME/g" | \
+ sed -r "s/mcast_join = 239.2.11.71//g" | sed -r "s/bind = 239.2.11.71//g" | \
+ sed -r "s/port = 8649/port = $2/g" | sed -r "s/retry_bind = true//g" > /opt/gmond-${1}.conf
+
+ chmod a+r /opt/gmond-${1}.conf
+
+ rm -f /opt/gmond-default.conf
+}
+
+# Downloads and setup Ganglia (and dependency) packages
+setupGangliaPackages()
+{
+ installGangliaPackages "master"
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ echo "data_source \"cassandra\" ${HOST_NAME}:8641" > /opt/gmetad.conf
+ echo "data_source \"ignite\" ${HOST_NAME}:8642" >> /opt/gmetad.conf
+ echo "data_source \"test\" ${HOST_NAME}:8643" >> /opt/gmetad.conf
+ #echo "data_source \"ganglia\" ${HOST_NAME}:8644" >> /opt/gmetad.conf
+ echo "setuid_username \"nobody\"" >> /opt/gmetad.conf
+ echo "case_sensitive_hostnames 0" >> /opt/gmetad.conf
+
+ chmod a+r /opt/gmetad.conf
+
+ createGmondReceiverConfig cassandra 8641
+ createGmondReceiverConfig ignite 8642
+ createGmondReceiverConfig test 8643
+ #createGmondSenderReceiverConfig ganglia 8644
+}
+
+# Starts 'gmond' receiver damon
+startGmondReceiver()
+{
+ configFile=/opt/gmond-${1}.conf
+ pidFile=/opt/gmond-${1}.pid
+
+ echo "[INFO] Starting gmond receiver daemon for $1 cluster using config file: $configFile"
+
+ rm -f $pidFile
+
+ /usr/local/sbin/gmond --conf=$configFile --pid-file=$pidFile
+
+ sleep 2s
+
+ if [ ! -f "$pidFile" ]; then
+ terminate "Failed to start gmond daemon for $1 cluster, pid file doesn't exist"
+ fi
+
+ pid=$(cat $pidFile)
+
+ echo "[INFO] gmond daemon for $1 cluster started, pid=$pid"
+
+ exists=$(ps $pid | grep gmond)
+
+ if [ -z "$exists" ]; then
+ terminate "gmond daemon for $1 cluster abnormally terminated"
+ fi
+}
+
+# Starts 'gmetad' daemon
+startGmetadCollector()
+{
+ echo "[INFO] Starting gmetad daemon"
+
+ rm -f /opt/gmetad.pid
+
+ /usr/local/sbin/gmetad --conf=/opt/gmetad.conf --pid-file=/opt/gmetad.pid
+
+ sleep 2s
+
+ if [ ! -f "/opt/gmetad.pid" ]; then
+ terminate "Failed to start gmetad daemon, pid file doesn't exist"
+ fi
+
+ pid=$(cat /opt/gmetad.pid)
+
+ echo "[INFO] gmetad daemon started, pid=$pid"
+
+ exists=$(ps $pid | grep gmetad)
+
+ if [ -z "$exists" ]; then
+ terminate "gmetad daemon abnormally terminated"
+ fi
+}
+
+# Starts Apache 'httpd' service
+startHttpdService()
+{
+ echo "[INFO] Starting httpd service"
+
+ service httpd start
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to start httpd service"
+ fi
+
+ sleep 5s
+
+ exists=$(service httpd status | grep running)
+ if [ -z "$exists" ]; then
+ terminate "httpd service process terminated"
+ fi
+
+ echo "[INFO] httpd service successfully started"
+}
+
+###################################################################################################################
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Ganglia master server"
+echo "[INFO]-----------------------------------------------------------------"
+
+setupPreRequisites
+setupJava
+setupAWSCLI
+setupTestsPackage
+setupGangliaPackages
+
+registerNode
+
+startGmondReceiver cassandra
+startGmondReceiver ignite
+startGmondReceiver test
+#startGmondReceiver ganglia
+startGmetadCollector
+startHttpdService
+
+terminate
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh
new file mode 100644
index 000000000..7f97ea1b7
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-bootstrap.sh
@@ -0,0 +1,336 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Bootstrap script to spin up Ignite cluster
+# -----------------------------------------------------------------------------------------------
+
+# URL to download AWS CLI tools
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+# URL to download JDK
+JDK_DOWNLOAD_URL=http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz
+
+# URL to download Ignite-Cassandra tests package - you should previously package and upload it to this place
+TESTS_PACKAGE_DONLOAD_URL=s3:////ignite-cassandra-tests-.zip
+
+# Terminates script execution and upload logs to S3
+terminate()
+{
+ SUCCESS_URL=$S3_IGNITE_BOOTSTRAP_SUCCESS
+ FAILURE_URL=$S3_IGNITE_BOOTSTRAP_FAILURE
+
+ if [ -n "$SUCCESS_URL" ] && [[ "$SUCCESS_URL" != */ ]]; then
+ SUCCESS_URL=${SUCCESS_URL}/
+ fi
+
+ if [ -n "$FAILURE_URL" ] && [[ "$FAILURE_URL" != */ ]]; then
+ FAILURE_URL=${FAILURE_URL}/
+ fi
+
+ host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ msg=$host_name
+
+ if [ -n "$1" ]; then
+ echo "[ERROR] $1"
+ echo "[ERROR]-----------------------------------------------------"
+ echo "[ERROR] Ignite node bootstrap failed"
+ echo "[ERROR]-----------------------------------------------------"
+ msg=$1
+
+ if [ -z "$FAILURE_URL" ]; then
+ exit 1
+ fi
+
+ reportFolder=${FAILURE_URL}${host_name}
+ reportFile=$reportFolder/__error__
+ else
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Ignite node bootstrap successfully completed"
+ echo "[INFO]-----------------------------------------------------"
+
+ if [ -z "$SUCCESS_URL" ]; then
+ exit 0
+ fi
+
+ reportFolder=${SUCCESS_URL}${host_name}
+ reportFile=$reportFolder/__success__
+ fi
+
+ echo $msg > /opt/bootstrap-result
+
+ aws s3 rm --recursive $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop report folder: $reportFolder"
+ fi
+
+ aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+ fi
+
+ rm -f /opt/bootstrap-result
+
+ if [ -n "$1" ]; then
+ exit 1
+ fi
+
+ exit 0
+}
+
+# Downloads specified package
+downloadPackage()
+{
+ echo "[INFO] Downloading $3 package from $1 into $2"
+
+ for i in 0 9;
+ do
+ if [[ "$1" == s3* ]]; then
+ aws s3 cp $1 $2
+ code=$?
+ else
+ curl "$1" -o "$2"
+ code=$?
+ fi
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] $3 package successfully downloaded from $1 into $2"
+ return 0
+ fi
+
+ echo "[WARN] Failed to download $3 package from $i attempt, sleeping extra 5sec"
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to download $3 package from $1 are failed"
+}
+
+# Downloads and setup JDK
+setupJava()
+{
+ rm -Rf /opt/java /opt/jdk.tar.gz
+
+ echo "[INFO] Downloading 'jdk'"
+ wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "$JDK_DOWNLOAD_URL" -O /opt/jdk.tar.gz
+ if [ $? -ne 0 ]; then
+ terminate "Failed to download 'jdk'"
+ fi
+
+ echo "[INFO] Untaring 'jdk'"
+ tar -xvzf /opt/jdk.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar 'jdk'"
+ fi
+
+ rm -Rf /opt/jdk.tar.gz
+
+ unzipDir=$(ls /opt | grep "jdk")
+ if [ "$unzipDir" != "java" ]; then
+ mv /opt/$unzipDir /opt/java
+ fi
+}
+
+# Downloads and setup AWS CLI
+setupAWSCLI()
+{
+ echo "[INFO] Installing 'awscli'"
+ pip install --upgrade awscli
+ if [ $? -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[ERROR] Failed to install 'awscli' using pip"
+ echo "[INFO] Trying to install awscli using zip archive"
+ echo "[INFO] Downloading awscli zip"
+
+ downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+ echo "[INFO] Unzipping awscli zip"
+ unzip /opt/awscli-bundle.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip awscli zip"
+ fi
+
+ rm -Rf /opt/awscli-bundle.zip
+
+ echo "[INFO] Installing awscli"
+ /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install awscli"
+ fi
+
+ echo "[INFO] Successfully installed awscli from zip archive"
+}
+
+# Setup all the pre-requisites (packages, settings and etc.)
+setupPreRequisites()
+{
+ echo "[INFO] Installing 'wget' package"
+ yum -y install wget
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'wget' package"
+ fi
+
+ echo "[INFO] Installing 'net-tools' package"
+ yum -y install net-tools
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'net-tools' package"
+ fi
+
+ echo "[INFO] Installing 'python' package"
+ yum -y install python
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'python' package"
+ fi
+
+ echo "[INFO] Installing 'unzip' package"
+ yum -y install unzip
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'unzip' package"
+ fi
+
+ downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+ echo "[INFO] Installing 'pip'"
+ python /opt/get-pip.py
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'pip'"
+ fi
+}
+
+# Downloads and setup tests package
+setupTestsPackage()
+{
+ downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/ignite-cassandra-tests.zip" "Tests"
+
+ rm -Rf /opt/ignite-cassandra-tests
+
+ unzip /opt/ignite-cassandra-tests.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip tests package"
+ fi
+
+ rm -f /opt/ignite-cassandra-tests.zip
+
+ unzipDir=$(ls /opt | grep "ignite-cassandra")
+ if [ "$unzipDir" != "ignite-cassandra-tests" ]; then
+ mv /opt/$unzipDir /opt/ignite-cassandra-tests
+ fi
+
+ find /opt/ignite-cassandra-tests -type f -name "*.sh" -exec chmod ug+x {} \;
+
+ . /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "ignite"
+
+ setupNTP
+
+ echo "[INFO] Starting logs collector daemon"
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ /opt/ignite-cassandra-tests/bootstrap/aws/logs-collector.sh "$S3_LOGS_TRIGGER" "$S3_IGNITE_LOGS/$HOST_NAME" "/opt/ignite/work/log" "/opt/ignite/ignite-start.log" > /opt/logs-collector.log &
+
+ echo "[INFO] Logs collector daemon started: $!"
+
+ echo "----------------------------------------------------------------------------------------"
+ printInstanceInfo
+ echo "----------------------------------------------------------------------------------------"
+ tagInstance
+ bootstrapGangliaAgent "ignite" 8642
+}
+
+# Downloads Ignite package
+downloadIgnite()
+{
+ downloadPackage "$IGNITE_DOWNLOAD_URL" "/opt/ignite.zip" "Ignite"
+
+ rm -Rf /opt/ignite
+
+ echo "[INFO] Unzipping Ignite package"
+ unzip /opt/ignite.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip Ignite package"
+ fi
+
+ rm -f /opt/ignite.zip
+
+ unzipDir=$(ls /opt | grep "ignite" | grep "apache")
+ if [ "$unzipDir" != "ignite" ]; then
+ mv /opt/$unzipDir /opt/ignite
+ fi
+}
+
+# Setups Ignite
+setupIgnite()
+{
+ echo "[INFO] Creating 'ignite' group"
+ exists=$(cat /etc/group | grep ignite)
+ if [ -z "$exists" ]; then
+ groupadd ignite
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'ignite' group"
+ fi
+ fi
+
+ echo "[INFO] Creating 'ignite' user"
+ exists=$(cat /etc/passwd | grep ignite)
+ if [ -z "$exists" ]; then
+ useradd -g ignite ignite
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'ignite' user"
+ fi
+ fi
+
+ testsJar=$(find /opt/ignite-cassandra-tests -type f -name "*.jar" | grep ignite-cassandra- | grep tests.jar)
+ if [ -n "$testsJar" ]; then
+ echo "[INFO] Coping tests jar $testsJar into /opt/ignite/libs/optional/ignite-cassandra"
+ cp $testsJar /opt/ignite/libs/optional/ignite-cassandra
+ if [ $? -ne 0 ]; then
+ terminate "Failed copy $testsJar into /opt/ignite/libs/optional/ignite-cassandra"
+ fi
+ fi
+
+ rm -f /opt/ignite/config/ignite-cassandra-server-template.xml
+ mv -f /opt/ignite-cassandra-tests/bootstrap/aws/ignite/ignite-cassandra-server-template.xml /opt/ignite/config
+
+ chown -R ignite:ignite /opt/ignite /opt/ignite-cassandra-tests
+
+ echo "export JAVA_HOME=/opt/java" >> $1
+ echo "export IGNITE_HOME=/opt/ignite" >> $1
+ echo "export USER_LIBS=\$IGNITE_HOME/libs/optional/ignite-cassandra/*:\$IGNITE_HOME/libs/optional/ignite-slf4j/*" >> $1
+ echo "export PATH=\$JAVA_HOME/bin:\$IGNITE_HOME/bin:\$PATH" >> $1
+}
+
+###################################################################################################################
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Ignite node"
+echo "[INFO]-----------------------------------------------------------------"
+
+setupPreRequisites
+setupJava
+setupAWSCLI
+setupTestsPackage
+
+downloadIgnite
+setupIgnite "/root/.bash_profile"
+
+cmd="/opt/ignite-cassandra-tests/bootstrap/aws/ignite/ignite-start.sh"
+
+#sudo -u ignite -g ignite sh -c "$cmd | tee /opt/ignite/ignite-start.log"
+
+$cmd | tee /opt/ignite/ignite-start.log
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml
new file mode 100644
index 000000000..692cd8b0b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-cassandra-server-template.xml
@@ -0,0 +1,181 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${CASSANDRA_SEEDS}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+ REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3}
+ AND DURABLE_WRITES = true
+
+
+ comment = 'A most excellent and useful table'
+ AND read_repair_chance = 0.2
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${IGNITE_SEEDS}
+
+
+
+
+
+
+
+
+
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-env.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-env.sh
new file mode 100644
index 000000000..bfe337191
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-env.sh
@@ -0,0 +1,29 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Environment setup script from Ignite
+# -----------------------------------------------------------------------------------------------
+
+JVM_OPTS="-Xms10g -Xmx10g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:+UseTLAB -XX:NewSize=128m -XX:MaxNewSize=768m"
+#JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=0 -XX:SurvivorRatio=1024 -XX:+UseCMSInitiatingOccupancyOnly -XX:CMSInitiatingOccupancyFraction=60"
+JVM_OPTS="$JVM_OPTS -Xss16m"
+
+export JVM_OPTS
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-start.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-start.sh
new file mode 100644
index 000000000..f2c15574a
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/ignite/ignite-start.sh
@@ -0,0 +1,266 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Script to start Ignite daemon (used by ignite-bootstrap.sh)
+# -----------------------------------------------------------------------------------------------
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+. /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "ignite"
+
+# Setups Cassandra seeds for this Ignite node being able to connect to Cassandra.
+# Looks for the information in S3 about already up and running Cassandra cluster nodes.
+setupCassandraSeeds()
+{
+ setupClusterSeeds "cassandra" "true"
+
+ CLUSTER_SEEDS=($CLUSTER_SEEDS)
+ count=${#CLUSTER_SEEDS[@]}
+
+ CASSANDRA_SEEDS=
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ seed=${CLUSTER_SEEDS[$i]}
+ CASSANDRA_SEEDS="${CASSANDRA_SEEDS}$seed<\/value>"
+ done
+
+ cat /opt/ignite/config/ignite-cassandra-server-template.xml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS/g" > /opt/ignite/config/ignite-cassandra-server.xml
+}
+
+# Setups Ignite nodes which this EC2 Ignite node will use to send its metadata and join Ignite cluster
+setupIgniteSeeds()
+{
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ echo "[INFO] Setting up Ignite seeds"
+
+ CLUSTER_SEEDS="127.0.0.1:47500..47509"
+
+ echo "[INFO] Using localhost address as a seed for the first Ignite node: $CLUSTER_SEEDS"
+
+ aws s3 rm --recursive ${S3_IGNITE_NODES_DISCOVERY::-1}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to clean Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY"
+ fi
+ else
+ setupClusterSeeds "ignite" "true"
+ fi
+
+ CLUSTER_SEEDS=($CLUSTER_SEEDS)
+ count=${#CLUSTER_SEEDS[@]}
+
+ IGNITE_SEEDS=
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ seed=${CLUSTER_SEEDS[$i]}
+ IGNITE_SEEDS="${IGNITE_SEEDS}$seed<\/value>"
+ done
+
+ cat /opt/ignite/config/ignite-cassandra-server.xml | sed -r "s/\\\$\{IGNITE_SEEDS\}/$IGNITE_SEEDS/g" > /opt/ignite/config/ignite-cassandra-server1.xml
+ mv -f /opt/ignite/config/ignite-cassandra-server1.xml /opt/ignite/config/ignite-cassandra-server.xml
+}
+
+# Checks status of Ignite daemon
+checkIgniteStatus()
+{
+ proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+
+ nodeId=
+ nodeAddrs=
+ nodePorts=
+ topology=
+ metrics=
+
+ logFile=$(ls /opt/ignite/work/log/ | grep "\.log$")
+ if [ -n "$logFile" ]; then
+ logFile=/opt/ignite/work/log/$logFile
+ nodeId=$(cat $logFile | grep "Local node \[ID")
+ nodeAddrs=$(cat $logFile | grep "Local node addresses:")
+ nodePorts=$(cat $logFile | grep "Local ports:")
+ topology=$(cat $logFile | grep "Topology snapshot")
+ metrics=$(cat $logFile | grep "Metrics for local node" | head -n 1)
+ fi
+
+ if [ -n "$nodeId" ] && [ -n "$nodeAddrs" ] && [ -n "$nodePorts" ] && [ -n "$topology" ] && [ -n "$metrics" ] && [ -n "$proc" ]; then
+ sleep 30s
+ return 0
+ fi
+
+ return 1
+}
+
+# Gracefully starts Ignite daemon and waits until it joins Ignite cluster
+startIgnite()
+{
+ echo "[INFO]-------------------------------------------------------------"
+ echo "[INFO] Trying attempt $START_ATTEMPT to start Ignite daemon"
+ echo "[INFO]-------------------------------------------------------------"
+ echo ""
+
+ setupCassandraSeeds
+ setupIgniteSeeds
+
+ waitToJoinCluster
+
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ aws s3 rm --recursive ${S3_IGNITE_NODES_DISCOVERY::-1}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to clean Ignite node discovery URL: $S3_IGNITE_NODES_DISCOVERY"
+ fi
+ fi
+
+ proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+ proc=($proc)
+
+ if [ -n "${proc[1]}" ]; then
+ echo "[INFO] Terminating existing Ignite process ${proc[1]}"
+ kill -9 ${proc[1]}
+ fi
+
+ echo "[INFO] Starting Ignite"
+ rm -Rf /opt/ignite/work/*
+ /opt/ignite/bin/ignite.sh /opt/ignite/config/ignite-cassandra-server.xml &
+
+ echo "[INFO] Ignite job id: $!"
+
+ sleep 1m
+
+ START_ATTEMPT=$(( $START_ATTEMPT+1 ))
+}
+
+#######################################################################################################
+
+START_ATTEMPT=0
+
+# Cleans all the previous metadata about this EC2 node
+unregisterNode
+
+# Tries to get first-node lock
+tryToGetFirstNodeLock
+
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ echo "[INFO] Starting first Ignite node"
+else
+ echo "[INFO] Starting Ignite node"
+fi
+
+echo "[INFO]-----------------------------------------------------------------"
+printInstanceInfo
+echo "[INFO]-----------------------------------------------------------------"
+
+if [ "$FIRST_NODE_LOCK" != "true" ]; then
+ waitFirstClusterNodeRegistered "true"
+else
+ cleanupMetadata
+fi
+
+# Applies Ignite environment settings from ignite-env.sh
+envScript=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/ignite-env.sh)
+if [ -f "$envScript" ]; then
+ . $envScript
+fi
+
+# Start Ignite daemon
+startIgnite
+
+startTime=$(date +%s)
+
+# Trying multiple attempts to start Ignite daemon
+while true; do
+ proc=$(ps -ef | grep java | grep "org.apache.ignite.startup.cmdline.CommandLineStartup")
+
+ checkIgniteStatus
+
+ if [ $? -eq 0 ]; then
+ sleep 1m
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Ignite daemon successfully started"
+ echo "[INFO]-----------------------------------------------------"
+ echo $proc
+ echo "[INFO]-----------------------------------------------------"
+
+ # Once node joined the cluster we need to remove cluster-join lock
+ # to allow other EC2 nodes to acquire it and join cluster sequentially
+ removeClusterJoinLock
+
+ break
+ fi
+
+ currentTime=$(date +%s)
+ duration=$(( $currentTime-$startTime ))
+ duration=$(( $duration/60 ))
+
+ if [ $duration -gt $SERVICE_STARTUP_TIME ]; then
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ # If the first node of Ignite cluster failed to start Ignite daemon in SERVICE_STARTUP_TIME min,
+ # we will not try any other attempts and just terminate with error. Terminate function itself, will
+ # take care about removing all the locks holding by this node.
+ terminate "${SERVICE_STARTUP_TIME}min timeout expired, but first Ignite daemon is still not up and running"
+ else
+ # If node isn't the first node of Ignite cluster and it failed to start we need to
+ # remove cluster-join lock to allow other EC2 nodes to acquire it
+ removeClusterJoinLock
+
+ # If node failed all SERVICE_START_ATTEMPTS attempts to start Ignite daemon we will not
+ # try anymore and terminate with error
+ if [ $START_ATTEMPT -gt $SERVICE_START_ATTEMPTS ]; then
+ terminate "${SERVICE_START_ATTEMPTS} attempts exceed, but Ignite daemon is still not up and running"
+ fi
+
+ # New attempt to start Ignite daemon
+ startIgnite
+ fi
+
+ continue
+ fi
+
+ # Handling situation when Ignite daemon process abnormally terminated
+ if [ -z "$proc" ]; then
+ # If this is the first node of Ignite cluster just terminating with error
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ terminate "Failed to start Ignite daemon"
+ fi
+
+ # Remove cluster-join lock to allow other EC2 nodes to acquire it
+ removeClusterJoinLock
+
+ echo "[WARN] Failed to start Ignite daemon. Sleeping for extra 30sec"
+ sleep 30s
+
+ # New attempt to start Ignite daemon
+ startIgnite
+
+ continue
+ fi
+
+ echo "[INFO] Waiting for Ignite daemon to start, time passed ${duration}min"
+ sleep 30s
+done
+
+# Once Ignite daemon successfully started we registering new Ignite node in S3
+registerNode
+
+# Terminating script with zero exit code
+terminate
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/logs-collector.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/logs-collector.sh
new file mode 100644
index 000000000..1634b89cf
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/logs-collector.sh
@@ -0,0 +1,173 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Logs collector daemon
+# -----------------------------------------------------------------------------------------------
+# Script is launched in background by all EC2 nodes of all clusters (Cassandra, Ignite, Tests) and
+# periodically (each 30 seconds) checks if specific S3 trigger file (specified by $S3_LOGS_TRIGGER_URL)
+# was created or its timestamp was changed. Such an event serve as a trigger for the script
+# to collect EC2 instance logs (from folder specified by $1) and upload them into specific
+# S3 folder (specified by $S3_LOGS_FOLDER).
+# -----------------------------------------------------------------------------------------------
+
+uploadLogs()
+{
+ if [ ! -d "$1" ]; then
+ echo "[INFO] Logs directory doesn't exist: $1"
+ return 0
+ fi
+
+ echo "[INFO] Uploading logs from directory: $1"
+
+ dirList=$(ls $1 | head -1)
+
+ if [ -z "$dirList" ]; then
+ echo "[INFO] Directory is empty: $1"
+ fi
+
+ for i in 0 9;
+ do
+ aws s3 sync --sse AES256 --delete "$1" "$S3_LOGS_FOLDER"
+ code=$?
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] Successfully uploaded logs from directory: $1"
+ return 0
+ fi
+
+ echo "[WARN] Failed to upload logs from $i attempt, sleeping extra 30sec"
+ sleep 30s
+ done
+
+ echo "[ERROR] All 10 attempts to upload logs are failed for the directory: $1"
+}
+
+createNewLogsSnapshot()
+{
+ rm -f ~/logs-collector.snapshot.new
+
+ for log_src in "$@"
+ do
+ if [ -d "$log_src" ] || [ -f "$log_src" ]; then
+ ls -alR $log_src >> ~/logs-collector.snapshot.new
+
+ fi
+ done
+}
+
+checkLogsChanged()
+{
+ createNewLogsSnapshot $@
+
+ if [ ! -f "~/logs-collector.snapshot" ]; then
+ return 1
+ fi
+
+ diff "~/logs-collector.snapshot" "~/logs-collector.snapshot.new" > /dev/null
+
+ return $?
+}
+
+updateLogsSnapshot()
+{
+ if [ ! -f "~/logs-collector.snapshot.new" ]; then
+ return 0
+ fi
+
+ rm -f "~/logs-collector.snapshot"
+ mv "~/logs-collector.snapshot.new" "~/logs-collector.snapshot"
+}
+
+collectLogs()
+{
+ createNewLogsSnapshot
+
+ rm -Rf ~/logs-collector-logs
+ mkdir -p ~/logs-collector-logs
+
+ for log_src in "$@"
+ do
+ if [ -f "$log_src" ]; then
+ echo "[INFO] Collecting log file: $log_src"
+ cp -f $log_src ~/logs-collector-logs
+ elif [ -d "$log_src" ]; then
+ echo "[INFO] Collecting logs from folder: $log_src"
+ cp -Rf $log_src ~/logs-collector-logs
+ fi
+ done
+
+ uploadLogs ~/logs-collector-logs
+
+ rm -Rf ~/logs-collector-logs
+
+ updateLogsSnapshot
+}
+
+echo "[INFO] Running Logs collector service"
+
+if [ -z "$1" ]; then
+ echo "[ERROR] Logs collection S3 trigger URL doesn't specified"
+ exit 1
+fi
+
+S3_LOGS_TRIGGER_URL=$1
+
+echo "[INFO] Logs collection S3 trigger URL: $S3_LOGS_TRIGGER_URL"
+
+if [ -z "$2" ]; then
+ echo "[ERROR] S3 folder where to upload logs doesn't specified"
+ exit 1
+fi
+
+S3_LOGS_FOLDER=$2
+
+echo "[INFO] S3 logs upload folder: $S3_LOGS_FOLDER"
+
+shift 2
+
+if [ -z "$1" ]; then
+ echo "[WARN] Local logs sources don't specified"
+else
+ echo "[INFO] Local logs sources: $@"
+fi
+
+echo "--------------------------------------------------------------------"
+
+TRIGGER_STATE=
+
+while true; do
+ sleep 30s
+
+ STATE=$(aws s3 ls $S3_LOGS_TRIGGER_URL)
+
+ if [ -z "$STATE" ] || [ "$STATE" == "$TRIGGER_STATE" ]; then
+ checkLogsChanged
+
+ if [ $? -eq 0 ]; then
+ continue
+ fi
+ fi
+
+ TRIGGER_STATE=$STATE
+
+ collectLogs $@ /var/log/cloud-init.log /var/log/cloud-init-output.log
+
+ echo "--------------------------------------------------------------------"
+done
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml
new file mode 100644
index 000000000..2989563dd
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/ignite-cassandra-client-template.xml
@@ -0,0 +1,183 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${CASSANDRA_SEEDS}
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+ REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3}
+ AND DURABLE_WRITES = true
+
+
+ comment = 'A most excellent and useful table'
+ AND read_repair_chance = 0.2
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+]]>
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ ${IGNITE_SEEDS}
+
+
+
+
+
+
+
+
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-bootstrap.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-bootstrap.sh
new file mode 100644
index 000000000..8e6faff69
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-bootstrap.sh
@@ -0,0 +1,317 @@
+#!/bin/sh
+
+#
+# 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.
+#
+
+# -----------------------------------------------------------------------------------------------
+# Bootstrap script to spin up Tests cluster
+# -----------------------------------------------------------------------------------------------
+
+# URL to download AWS CLI tools
+AWS_CLI_DOWNLOAD_URL=https://s3.amazonaws.com/aws-cli/awscli-bundle.zip
+
+# URL to download JDK
+JDK_DOWNLOAD_URL=http://download.oracle.com/otn-pub/java/jdk/8u77-b03/jdk-8u77-linux-x64.tar.gz
+
+# URL to download Ignite-Cassandra tests package - you should previously package and upload it to this place
+TESTS_PACKAGE_DONLOAD_URL=s3:////ignite-cassandra-tests-.zip
+
+# Terminates script execution and upload logs to S3
+terminate()
+{
+ SUCCESS_URL=$S3_TESTS_SUCCESS
+ FAILURE_URL=$S3_TESTS_FAILURE
+
+ if [ -n "$SUCCESS_URL" ] && [[ "$SUCCESS_URL" != */ ]]; then
+ SUCCESS_URL=${SUCCESS_URL}/
+ fi
+
+ if [ -n "$FAILURE_URL" ] && [[ "$FAILURE_URL" != */ ]]; then
+ FAILURE_URL=${FAILURE_URL}/
+ fi
+
+ host_name=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ msg=$host_name
+
+ if [ -n "$1" ]; then
+ echo "[ERROR] $1"
+ echo "[ERROR]-----------------------------------------------------"
+ echo "[ERROR] Test node bootstrap failed"
+ echo "[ERROR]-----------------------------------------------------"
+ msg=$1
+
+ if [ -z "$FAILURE_URL" ]; then
+ exit 1
+ fi
+
+ reportFolder=${FAILURE_URL}${host_name}
+ reportFile=$reportFolder/__error__
+ else
+ echo "[INFO]-----------------------------------------------------"
+ echo "[INFO] Test node bootstrap successfully completed"
+ echo "[INFO]-----------------------------------------------------"
+
+ if [ -z "$SUCCESS_URL" ]; then
+ exit 0
+ fi
+
+ reportFolder=${SUCCESS_URL}${host_name}
+ reportFile=$reportFolder/__success__
+ fi
+
+ echo $msg > /opt/bootstrap-result
+
+ aws s3 rm --recursive $reportFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop report folder: $reportFolder"
+ fi
+
+ aws s3 cp --sse AES256 /opt/bootstrap-result $reportFile
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to report bootstrap result to: $reportFile"
+ fi
+
+ rm -f /opt/bootstrap-result
+
+ if [ -n "$1" ]; then
+ exit 1
+ fi
+
+ exit 0
+}
+
+# Downloads specified package
+downloadPackage()
+{
+ echo "[INFO] Downloading $3 package from $1 into $2"
+
+ for i in 0 9;
+ do
+ if [[ "$1" == s3* ]]; then
+ aws s3 cp $1 $2
+ code=$?
+ else
+ curl "$1" -o "$2"
+ code=$?
+ fi
+
+ if [ $code -eq 0 ]; then
+ echo "[INFO] $3 package successfully downloaded from $1 into $2"
+ return 0
+ fi
+
+ echo "[WARN] Failed to download $3 package from $i attempt, sleeping extra 5sec"
+ sleep 5s
+ done
+
+ terminate "All 10 attempts to download $3 package from $1 are failed"
+}
+
+# Downloads and setup JDK
+setupJava()
+{
+ rm -Rf /opt/java /opt/jdk.tar.gz
+
+ echo "[INFO] Downloading 'jdk'"
+ wget --no-cookies --no-check-certificate --header "Cookie: gpw_e24=http%3A%2F%2Fwww.oracle.com%2F; oraclelicense=accept-securebackup-cookie" "$JDK_DOWNLOAD_URL" -O /opt/jdk.tar.gz
+ if [ $? -ne 0 ]; then
+ terminate "Failed to download 'jdk'"
+ fi
+
+ echo "[INFO] Untaring 'jdk'"
+ tar -xvzf /opt/jdk.tar.gz -C /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to untar 'jdk'"
+ fi
+
+ rm -Rf /opt/jdk.tar.gz
+
+ unzipDir=$(ls /opt | grep "jdk")
+ if [ "$unzipDir" != "java" ]; then
+ mv /opt/$unzipDir /opt/java
+ fi
+}
+
+# Downloads and setup AWS CLI
+setupAWSCLI()
+{
+ echo "[INFO] Installing 'awscli'"
+ pip install --upgrade awscli
+ if [ $? -eq 0 ]; then
+ return 0
+ fi
+
+ echo "[ERROR] Failed to install 'awscli' using pip"
+ echo "[INFO] Trying to install awscli using zip archive"
+ echo "[INFO] Downloading awscli zip"
+
+ downloadPackage "$AWS_CLI_DOWNLOAD_URL" "/opt/awscli-bundle.zip" "awscli"
+
+ echo "[INFO] Unzipping awscli zip"
+ unzip /opt/awscli-bundle.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip awscli zip"
+ fi
+
+ rm -Rf /opt/awscli-bundle.zip
+
+ echo "[INFO] Installing awscli"
+ /opt/awscli-bundle/install -i /usr/local/aws -b /usr/local/bin/aws
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install awscli"
+ fi
+
+ echo "[INFO] Successfully installed awscli from zip archive"
+}
+
+# Setup all the pre-requisites (packages, settings and etc.)
+setupPreRequisites()
+{
+ echo "[INFO] Installing 'wget' package"
+ yum -y install wget
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'wget' package"
+ fi
+
+ echo "[INFO] Installing 'net-tools' package"
+ yum -y install net-tools
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'net-tools' package"
+ fi
+
+ echo "[INFO] Installing 'python' package"
+ yum -y install python
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'python' package"
+ fi
+
+ echo "[INFO] Installing 'unzip' package"
+ yum -y install unzip
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'unzip' package"
+ fi
+
+ downloadPackage "https://bootstrap.pypa.io/get-pip.py" "/opt/get-pip.py" "get-pip.py"
+
+ echo "[INFO] Installing 'pip'"
+ python /opt/get-pip.py
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install 'pip'"
+ fi
+}
+
+# Downloads and setup tests package
+setupTestsPackage()
+{
+ downloadPackage "$TESTS_PACKAGE_DONLOAD_URL" "/opt/ignite-cassandra-tests.zip" "Tests"
+
+ rm -Rf /opt/ignite-cassandra-tests
+
+ unzip /opt/ignite-cassandra-tests.zip -d /opt
+ if [ $? -ne 0 ]; then
+ terminate "Failed to unzip tests package"
+ fi
+
+ rm -f /opt/ignite-cassandra-tests.zip
+
+ unzipDir=$(ls /opt | grep "ignite-cassandra")
+ if [ "$unzipDir" != "ignite-cassandra-tests" ]; then
+ mv /opt/$unzipDir /opt/ignite-cassandra-tests
+ fi
+
+ find /opt/ignite-cassandra-tests -type f -name "*.sh" -exec chmod ug+x {} \;
+
+ . /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "test"
+
+ setupNTP
+
+ echo "[INFO] Starting logs collector daemon"
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+ /opt/ignite-cassandra-tests/bootstrap/aws/logs-collector.sh "$S3_LOGS_TRIGGER" "$S3_TESTS_LOGS/$HOST_NAME" "/opt/ignite-cassandra-tests/logs" > /opt/logs-collector.log &
+
+ echo "[INFO] Logs collector daemon started: $!"
+
+ echo "----------------------------------------------------------------------------------------"
+ printInstanceInfo
+ echo "----------------------------------------------------------------------------------------"
+ tagInstance
+ bootstrapGangliaAgent "test" 8643
+
+ ###################################################
+ # Extra configuration specific only for test node #
+ ###################################################
+
+ echo "[INFO] Installing bc package"
+
+ yum -y install bc
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install bc package"
+ fi
+
+ echo "[INFO] Installing zip package"
+
+ yum -y install zip
+
+ if [ $? -ne 0 ]; then
+ terminate "Failed to install zip package"
+ fi
+
+ echo "[INFO] Creating 'ignite' group"
+ exists=$(cat /etc/group | grep ignite)
+ if [ -z "$exists" ]; then
+ groupadd ignite
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'ignite' group"
+ fi
+ fi
+
+ echo "[INFO] Creating 'ignite' user"
+ exists=$(cat /etc/passwd | grep ignite)
+ if [ -z "$exists" ]; then
+ useradd -g ignite ignite
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create 'ignite' user"
+ fi
+ fi
+
+ mkdir -p /opt/ignite-cassandra-tests/logs
+ chown -R ignite:ignite /opt/ignite-cassandra-tests
+
+ echo "export JAVA_HOME=/opt/java" >> $1
+ echo "export PATH=\$JAVA_HOME/bin:\$PATH" >> $1
+}
+
+###################################################################################################################
+
+echo "[INFO]-----------------------------------------------------------------"
+echo "[INFO] Bootstrapping Tests node"
+echo "[INFO]-----------------------------------------------------------------"
+
+setupPreRequisites
+setupJava
+setupAWSCLI
+setupTestsPackage "/root/.bash_profile"
+
+cmd="/opt/ignite-cassandra-tests/bootstrap/aws/tests/tests-manager.sh"
+
+#sudo -u ignite -g ignite sh -c "$cmd > /opt/ignite-cassandra-tests/tests-manager" &
+
+$cmd > /opt/ignite-cassandra-tests/logs/tests-manager.log &
+
+terminate
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-manager.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-manager.sh
new file mode 100644
index 000000000..c0f5d6b8c
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-manager.sh
@@ -0,0 +1,458 @@
+#!/bin/sh
+
+#
+# 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 manager daemon
+# -----------------------------------------------------------------------------------------------
+# Script is launched in background by all nodes of Tests cluster and
+# periodically (each 30 seconds) checks if specific S3 trigger file was created or
+# its timestamp was changed. Such an event serve as a trigger for the script to start
+# preparing to run load tests.
+# -----------------------------------------------------------------------------------------------
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+. /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "test"
+
+# Switch test node to IDLE state
+switchToIdleState()
+{
+ if [ "$NODE_STATE" != "IDLE" ]; then
+ echo "[INFO] Switching node to IDLE state"
+ dropStateFlag "$S3_TESTS_WAITING" "$S3_TESTS_PREPARING" "$S3_TESTS_RUNNING"
+ createStateFlag "$S3_TESTS_IDLE"
+ NODE_STATE="IDLE"
+ echo "[INFO] Node was switched to IDLE state"
+ fi
+}
+
+# Switch test node to PREPARING state
+switchToPreparingState()
+{
+ if [ "$NODE_STATE" != "PREPARING" ]; then
+ echo "[INFO] Switching node to PREPARING state"
+ dropStateFlag "$S3_TESTS_WAITING" "$S3_TESTS_IDLE" "$S3_TESTS_RUNNING"
+ createStateFlag "$S3_TESTS_PREPARING"
+ NODE_STATE="PREPARING"
+ echo "[INFO] Node was switched to PREPARING state"
+ fi
+}
+
+# Switch test node to WAITING state
+switchToWaitingState()
+{
+ if [ "$NODE_STATE" != "WAITING" ]; then
+ echo "[INFO] Switching node to WAITING state"
+ dropStateFlag "$S3_TESTS_IDLE" "$S3_TESTS_PREPARING" "$S3_TESTS_RUNNING"
+ createStateFlag "$S3_TESTS_WAITING"
+ NODE_STATE="WAITING"
+ echo "[INFO] Node was switched to WAITING state"
+ fi
+}
+
+# Switch test node to RUNNING state
+switchToRunningState()
+{
+ if [ "$NODE_STATE" != "RUNNING" ]; then
+ echo "[INFO] Switching node to RUNNING state"
+ dropStateFlag "$S3_TESTS_IDLE" "$S3_TESTS_PREPARING" "$S3_TESTS_WAITING"
+ createStateFlag "$S3_TESTS_RUNNING"
+ NODE_STATE="RUNNING"
+ echo "[INFO] Node was switched to RUNNING state"
+ fi
+}
+
+# Creates appropriate state flag for the node in S3
+createStateFlag()
+{
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ aws s3 cp --sse AES256 /etc/hosts ${1}${HOST_NAME}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to create state flag: ${1}${HOST_NAME}"
+ fi
+}
+
+# Drops appropriate state flag for the node in S3
+dropStateFlag()
+{
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ for flagUrl in "$@"
+ do
+ exists=$(aws s3 ls ${flagUrl}${HOST_NAME})
+ if [ -n "$exists" ]; then
+ aws s3 rm ${flagUrl}${HOST_NAME}
+ if [ $? -ne 0 ]; then
+ terminate "Failed to drop state flag: ${flagUrl}${HOST_NAME}"
+ fi
+ fi
+ done
+}
+
+# Removes tests summary report from S3
+dropTestsSummary()
+{
+ exists=$(aws s3 ls $S3_TESTS_SUMMARY)
+ if [ -z "$exists" ]; then
+ return 0
+ fi
+
+ aws s3 rm $S3_TESTS_SUMMARY
+ if [ $? -ne 0 ]; then
+ terminate "Failed to drop tests summary info: $S3_TESTS_SUMMARY"
+ fi
+}
+
+# Recreate all the necessary Cassandra artifacts before running Load tests
+recreateCassandraArtifacts()
+{
+ /opt/ignite-cassandra-tests/recreate-cassandra-artifacts.sh
+ if [ $? -ne 0 ]; then
+ terminate "Failed to recreate Cassandra artifacts"
+ fi
+}
+
+# Setups Cassandra seeds for this Tests node being able to connect to Cassandra.
+# Looks for the information in S3 about already up and running Cassandra cluster nodes.
+setupCassandraSeeds()
+{
+ if [ $CASSANDRA_NODES_COUNT -eq 0 ]; then
+ return 0
+ fi
+
+ setupClusterSeeds "cassandra"
+
+ CASSANDRA_SEEDS1=$(echo $CLUSTER_SEEDS | sed -r "s/ /,/g")
+ CASSANDRA_SEEDS2=
+
+ CLUSTER_SEEDS=($CLUSTER_SEEDS)
+ count=${#CLUSTER_SEEDS[@]}
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ seed=${CLUSTER_SEEDS[$i]}
+ CASSANDRA_SEEDS2="${CASSANDRA_SEEDS2}$seed<\/value>"
+ done
+
+ echo "[INFO] Using Cassandra seeds: $CASSANDRA_SEEDS1"
+
+ echo "contact.points=$CASSANDRA_SEEDS1" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/connection.properties
+
+ cat /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template.xml | sed -r "s/\\\$\{CASSANDRA_SEEDS\}/$CASSANDRA_SEEDS2/g" > /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml
+}
+
+# Setups Ignite nodes for this Tests node being able to connect to Ignite.
+# Looks for the information in S3 about already up and running Cassandra cluster nodes.
+setupIgniteSeeds()
+{
+ if [ $IGNITE_NODES_COUNT -eq 0 ]; then
+ return 0
+ fi
+
+ setupClusterSeeds "ignite"
+
+ CLUSTER_SEEDS=($CLUSTER_SEEDS)
+ count=${#CLUSTER_SEEDS[@]}
+
+ IGNITE_SEEDS=
+
+ for (( i=0; i<=$(( $count -1 )); i++ ))
+ do
+ seed=${CLUSTER_SEEDS[$i]}
+ IGNITE_SEEDS="${IGNITE_SEEDS}$seed<\/value>"
+ done
+
+ echo "[INFO] Using Ignite seeds: $IGNITE_SEEDS"
+
+ cat /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml | sed -r "s/\\\$\{IGNITE_SEEDS\}/$IGNITE_SEEDS/g" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml
+ rm -f /opt/ignite-cassandra-tests/bootstrap/aws/tests/ignite-cassandra-client-template1.xml
+}
+
+# Setups Cassandra credentials to connect to Cassandra cluster
+setupCassandraCredentials()
+{
+ echo "admin.user=cassandra" > /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+ echo "admin.password=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+ echo "regular.user=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+ echo "regular.password=cassandra" >> /opt/ignite-cassandra-tests/settings/org/apache/ignite/tests/cassandra/credentials.properties
+}
+
+# Triggering first time tests execution for all nodes in the Tests cluster
+triggerFirstTimeTestsExecution()
+{
+ if [ -z "$TESTS_TYPE" ]; then
+ return 0
+ fi
+
+ tryToGetFirstNodeLock
+ if [ $? -ne 0 ]; then
+ return 0
+ fi
+
+ sleep 30s
+
+ echo "[INFO] Triggering first time tests execution"
+
+ echo "TESTS_TYPE=$TESTS_TYPE" > /opt/ignite-cassandra-tests/tests-trigger
+ echo "#--------------------------------------------------" >> /opt/ignite-cassandra-tests/tests-trigger
+ echo "" >> /opt/ignite-cassandra-tests/tests-trigger
+ cat /opt/ignite-cassandra-tests/settings/tests.properties >> /opt/ignite-cassandra-tests/tests-trigger
+
+ aws s3 cp --sse AES256 /opt/ignite-cassandra-tests/tests-trigger $S3_TESTS_TRIGGER
+ code=$?
+
+ rm -f /opt/ignite-cassandra-tests/tests-trigger
+
+ if [ $code -ne 0 ]; then
+ terminate "Failed to create tests trigger: $S3_TESTS_TRIGGER"
+ fi
+}
+
+# Cleans previously created logs from S3
+cleanPreviousLogs()
+{
+ for logFile in /opt/ignite-cassandra-tests/logs/*
+ do
+ managerLog=$(echo $logFile | grep "tests-manager")
+ if [ -z "$managerLog" ]; then
+ rm -Rf $logFile
+ fi
+ done
+
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ aws s3 rm --recursive ${S3_TESTS_FAILURE}${HOST_NAME}
+ aws s3 rm --recursive ${S3_TESTS_SUCCESS}${HOST_NAME}
+}
+
+# Uploads tests logs to S3
+uploadTestsLogs()
+{
+ HOST_NAME=$(hostname -f | tr '[:upper:]' '[:lower:]')
+
+ if [ -f "/opt/ignite-cassandra-tests/logs/__success__" ]; then
+ logsFolder=${S3_TESTS_SUCCESS}${HOST_NAME}
+ else
+ logsFolder=${S3_TESTS_FAILURE}${HOST_NAME}
+ fi
+
+ aws s3 rm --recursive $logsFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to drop logs folder: $logsFolder"
+ fi
+
+ if [ -d "/opt/ignite-cassandra-tests/logs" ]; then
+ aws s3 sync --sse AES256 /opt/ignite-cassandra-tests/logs $logsFolder
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to export tests logs to: $logsFolder"
+ fi
+ fi
+}
+
+# Runs tests-report.sh to prepare tests summary report
+buildTestsSummaryReport()
+{
+ reportScript=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )/tests-report.sh)
+ $reportScript
+
+ if [ -n "$S3_LOGS_TRIGGER" ]; then
+ aws s3 cp --sse AES256 /etc/hosts $S3_LOGS_TRIGGER
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to trigger logs collection"
+ fi
+ fi
+}
+
+# Running load tests
+runLoadTests()
+{
+ cd /opt/ignite-cassandra-tests
+
+ if [ "$TESTS_TYPE" == "ignite" ]; then
+ echo "[INFO] Running Ignite load tests"
+ ./ignite-load-tests.sh &
+ else
+ echo "[INFO] Running Cassandra load tests"
+ ./cassandra-load-tests.sh &
+ fi
+
+ testsJobId=$!
+
+ echo "[INFO] Tests job id: $testsJobId"
+
+ sleep 1m
+
+ LOGS_SNAPSHOT=$(ls -al /opt/ignite-cassandra-tests/logs)
+ LOGS_SNAPSHOT_TIME=$(date +%s)
+
+ TERMINATED=
+
+ # tests monitoring
+ while true; do
+ proc=$(ps -ef | grep java | grep "org.apache.ignite.tests")
+ if [ -z "$proc" ]; then
+ break
+ fi
+
+ NEW_LOGS_SNAPSHOT=$(ls -al /opt/ignite-cassandra-tests/logs)
+ NEW_LOGS_SNAPSHOT_TIME=$(date +%s)
+
+ # if logs state updated it means that tests are running and not stuck
+ if [ "$LOGS_SNAPSHOT" != "$NEW_LOGS_SNAPSHOT" ]; then
+ LOGS_SNAPSHOT=$NEW_LOGS_SNAPSHOT
+ LOGS_SNAPSHOT_TIME=$NEW_LOGS_SNAPSHOT_TIME
+ continue
+ fi
+
+ duration=$(( $NEW_LOGS_SNAPSHOT_TIME-$LOGS_SNAPSHOT_TIME ))
+ duration=$(( $duration/60 ))
+
+ # if logs wasn't updated during 5min it means that load tests stuck
+ if [ $duration -gt 5 ]; then
+ proc=($proc)
+ kill -9 ${proc[1]}
+ TERMINATED="true"
+ break
+ fi
+
+ echo "[INFO] Waiting extra 30sec for load tests to complete"
+
+ sleep 30s
+ done
+
+ rm -f /opt/ignite-cassandra-tests/logs/tests.properties
+ cp /opt/ignite-cassandra-tests/settings/tests.properties /opt/ignite-cassandra-tests/logs
+
+ if [ "$TERMINATED" == "true" ]; then
+ echo "[ERROR] Load tests stuck, tests process terminated"
+ echo "Load tests stuck, tests process terminated" > /opt/ignite-cassandra-tests/logs/__error__
+ return 0
+ fi
+
+ failed=
+ if [ "$TESTS_TYPE" == "cassandra" ]; then
+ failed=$(cat /opt/ignite-cassandra-tests/cassandra-load-tests.log | grep "load tests execution failed")
+ else
+ failed=$(cat /opt/ignite-cassandra-tests/ignite-load-tests.log | grep "load tests execution failed")
+ fi
+
+ if [ -n "$failed" ]; then
+ echo "[ERROR] Load tests execution failed"
+ echo "Load tests execution failed" > /opt/ignite-cassandra-tests/logs/__error__
+ else
+ echo "[INFO] Load tests execution successfully completed"
+ echo "Load tests execution successfully completed" > /opt/ignite-cassandra-tests/logs/__success__
+ fi
+}
+
+#######################################################################################################
+
+sleep 1m
+
+NODE_STATE=
+TRIGGER_STATE=
+
+printInstanceInfo
+setupCassandraCredentials
+switchToIdleState
+
+triggerFirstTimeTestsExecution
+
+registerNode
+
+while true; do
+ # switching state to IDLE
+ switchToIdleState
+
+ sleep 30s
+
+ NEW_TRIGGER_STATE=$(aws s3 ls $S3_TESTS_TRIGGER | xargs)
+ if [ -z "$NEW_TRIGGER_STATE" ] || [ "$NEW_TRIGGER_STATE" == "$TRIGGER_STATE" ]; then
+ continue
+ fi
+
+ echo "----------------------------------------------------------------------"
+ echo "[INFO] Tests trigger changed"
+ echo "----------------------------------------------------------------------"
+ echo "[INFO] Old trigger: $TRIGGER_STATE"
+ echo "----------------------------------------------------------------------"
+ echo "[INFO] New trigger: $NEW_TRIGGER_STATE"
+ echo "----------------------------------------------------------------------"
+
+ TRIGGER_STATE=$NEW_TRIGGER_STATE
+
+ aws s3 cp $S3_TESTS_TRIGGER /opt/ignite-cassandra-tests/tests-trigger
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to download tests trigger info from: $S3_TESTS_TRIGGER"
+ continue
+ fi
+
+ TESTS_TYPE=$(cat /opt/ignite-cassandra-tests/tests-trigger | grep TESTS_TYPE | xargs | sed -r "s/TESTS_TYPE=//g")
+ if [ "$TESTS_TYPE" != "ignite" ] && [ "$TESTS_TYPE" != "cassandra" ]; then
+ rm -f /opt/ignite-cassandra-tests/tests-trigger
+ echo "[ERROR] Incorrect tests type specified in the trigger info: $S3_TESTS_TRIGGER"
+ continue
+ fi
+
+ rm -f /opt/ignite-cassandra-tests/settings/tests.properties
+ mv -f /opt/ignite-cassandra-tests/tests-trigger /opt/ignite-cassandra-tests/settings/tests.properties
+
+ waitAllTestNodesCompletedTests
+
+ # switching state to PREPARING
+ switchToPreparingState
+
+ waitAllClusterNodesReady "cassandra"
+ waitAllClusterNodesReady "ignite"
+ setupCassandraSeeds
+ setupIgniteSeeds
+
+ cleanPreviousLogs
+
+ tryToGetFirstNodeLock
+ if [ $? -eq 0 ]; then
+ dropTestsSummary
+ recreateCassandraArtifacts
+ fi
+
+ # switching state to WAITING
+ switchToWaitingState
+
+ waitAllClusterNodesReady "test"
+
+ if [ "$FIRST_NODE_LOCK" == "true" ]; then
+ aws s3 rm $S3_TESTS_TRIGGER
+ fi
+
+ # switching state to RUNNING
+ switchToRunningState
+
+ runLoadTests
+ uploadTestsLogs
+
+ tryToGetFirstNodeLock
+ if [ $? -eq 0 ]; then
+ waitAllTestNodesCompletedTests
+ buildTestsSummaryReport
+ removeFirstNodeLock
+ fi
+done
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-report.sh b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-report.sh
new file mode 100644
index 000000000..1576d5737
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/bootstrap/aws/tests/tests-report.sh
@@ -0,0 +1,499 @@
+#!/bin/sh
+
+#
+# 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 report builder
+# -----------------------------------------------------------------------------------------------
+# Script is used to analyze load tests logs collected from all 'Tests' cluster nodes and build
+# summary report
+# -----------------------------------------------------------------------------------------------
+
+#profile=/home/ignite/.bash_profile
+profile=/root/.bash_profile
+
+. $profile
+. /opt/ignite-cassandra-tests/bootstrap/aws/common.sh "test"
+
+# Building tests summary report
+reportTestsSummary()
+{
+ echo "[INFO] Preparing tests results summary"
+
+ TESTS_SUMMARY_DIR=/opt/ignite-cassandra-tests/tests-summary
+ SUCCEED_NODES_FILE=$TESTS_SUMMARY_DIR/succeed-nodes
+ SUCCEED_NODES_DIR=$TESTS_SUMMARY_DIR/succeed
+ FAILED_NODES_FILE=$TESTS_SUMMARY_DIR/failed-nodes
+ FAILED_NODES_DIR=$TESTS_SUMMARY_DIR/failed
+ REPORT_FILE=$TESTS_SUMMARY_DIR/report.txt
+
+ rm -Rf $TESTS_SUMMARY_DIR
+ mkdir -p $TESTS_SUMMARY_DIR
+ mkdir -p $SUCCEED_NODES_DIR
+ mkdir -p $FAILED_NODES_DIR
+
+ aws s3 ls $S3_TESTS_SUCCESS | sed -r "s/PRE //g" | sed -r "s/ //g" | sed -r "s/\///g" > $SUCCEED_NODES_FILE
+ aws s3 ls $S3_TESTS_FAILURE | sed -r "s/PRE //g" | sed -r "s/ //g" | sed -r "s/\///g" > $FAILED_NODES_FILE
+
+ succeedCount=$(cat $SUCCEED_NODES_FILE | wc -l)
+ failedCount=$(cat $FAILED_NODES_FILE | wc -l)
+ count=$(( $succeedCount+$failedCount ))
+
+ echo "Test type : $TESTS_TYPE" > $REPORT_FILE
+ echo "Test nodes count : $count" >> $REPORT_FILE
+ echo "Test nodes succeed: $succeedCount" >> $REPORT_FILE
+ echo "Test nodes failed : $failedCount" >> $REPORT_FILE
+ echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+ if [ $succeedCount -gt 0 ]; then
+ echo "Succeed test nodes |" >> $REPORT_FILE
+ echo "-------------------" >> $REPORT_FILE
+ cat $SUCCEED_NODES_FILE >> $REPORT_FILE
+ echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+ aws s3 sync --delete $S3_TESTS_SUCCESS $SUCCEED_NODES_DIR
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to get succeed tests details"
+ else
+ reportSucceedTestsStatistics "$REPORT_FILE" "$SUCCEED_NODES_DIR"
+ fi
+ fi
+
+ if [ $failedCount -gt 0 ]; then
+ echo "Failed test nodes |" >> $REPORT_FILE
+ echo "------------------" >> $REPORT_FILE
+ cat $FAILED_NODES_FILE >> $REPORT_FILE
+ echo "----------------------------------------------------------------------------------------------" >> $REPORT_FILE
+
+ aws sync --delete $S3_TESTS_FAILURE $FAILED_NODES_DIR
+ if [ $? -ne 0 ]; then
+ echo "[ERROR] Failed to get failed tests details"
+ else
+ reportFailedTestsDetailes "$REPORT_FILE" "$FAILED_NODES_DIR"
+ fi
+ fi
+
+ rm -f $HOME/tests-summary.zip
+
+ pushd $TESTS_SUMMARY_DIR
+
+ zip -r -9 $HOME/tests-summary.zip .
+ code=$?
+
+ rm -Rf $TESTS_SUMMARY_DIR
+
+ popd
+
+ if [ $code -ne 0 ]; then
+ echo "-------------------------------------------------------------------------------------"
+ echo "[ERROR] Failed to create tests summary zip archive $HOME/tests-summary.zip for $TESTS_SUMMARY_DIR"
+ echo "-------------------------------------------------------------------------------------"
+ return 1
+ fi
+
+ aws s3 cp --sse AES256 $HOME/tests-summary.zip $S3_TESTS_SUMMARY
+ if [ $? -ne 0 ]; then
+ echo "-------------------------------------------------------------------------------------"
+ echo "[ERROR] Failed to uploat tests summary archive to: $S3_TESTS_SUMMARY"
+ echo "-------------------------------------------------------------------------------------"
+ else
+ echo "-------------------------------------------------------------------------------------"
+ echo "[INFO] Tests results summary uploaded to: $S3_TESTS_SUMMARY"
+ echo "-------------------------------------------------------------------------------------"
+ fi
+
+ rm -f $HOME/tests-summary.zip
+}
+
+# Creates report for succeed tests
+reportSucceedTestsStatistics()
+{
+ writeMsg="0"
+ writeErrors="0"
+ writeSpeed="0"
+ blkWriteMsg="0"
+ blkWriteErrors="0"
+ blkWriteSpeed="0"
+ readMsg="0"
+ readErrors="0"
+ readSpeed="0"
+ blkReadMsg="0"
+ blkReadErrors="0"
+ blkReadSpeed="0"
+
+ writeErrNodes=
+ blkWriteErrNodes=
+ readErrNodes=
+ blkReadErrNodes=
+
+ tmpFile=`mktemp`
+
+ for dir in $2/*
+ do
+ node=$(echo $dir | sed -r "s/^.*\///g")
+ echo "-------------------------------------------------------------------------------------"
+ echo "[INFO] Gathering statistics from $node test node"
+ echo "-------------------------------------------------------------------------------------"
+
+ logFile=$(ls $dir | grep "${TESTS_TYPE}-load-tests.log" | head -1)
+ if [ -z "$logFile" ]; then
+ echo "[WARN] Node $node marked as succeeded, but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file"
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Node $node marked as succeeded," >> $tmpFile
+ echo "but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file" >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ continue
+ fi
+
+ logFile=$dir/$logFile
+ if [ ! -f "$logFile" ]; then
+ echo "[WARN] Node $node marked as succeeded, but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file"
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Node $node marked as succeeded," >> $tmpFile
+ echo "but it doesn't have \"${TESTS_TYPE}-load-tests.log\" tests results summary file" >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ continue
+ fi
+
+ cnt=$(cat $logFile | grep "^WRITE messages" | sed -r "s/WRITE messages: //g" | xargs)
+ if [ -n "$cnt" ]; then
+ writeMsg=$(bc <<< "$writeMsg + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] WRITE messages: $cnt"
+ else
+ echo "[WARN] WRITE messages count is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "WRITE messages count is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect WRITE messages count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect WRITE messages count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^WRITE errors" | sed -r "s/WRITE errors: //g" | sed -r "s/,.*//g" | xargs)
+ if [ -n "$cnt" ]; then
+ echo "[INFO] WRITE errors: $cnt"
+ writeErrors=$(bc <<< "$writeErrors + $cnt")
+ if [ $cnt -ne 0 ]; then
+ if [ -n "$writeErrNodes" ]; then
+ writeErrNodes="${writeErrNodes}, "
+ fi
+ writeErrNodes="${writeErrNodes}${node}"
+ fi
+ else
+ echo "[WARN] Failed to detect WRITE errors count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect WRITE errors count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^WRITE speed" | sed -r "s/WRITE speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+ if [ -n "$cnt" ]; then
+ writeSpeed=$(bc <<< "$writeSpeed + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] WRITE speed: $cnt msg/sec"
+ else
+ echo "[WARN] WRITE speed is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "WRITE speed is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect WRITE speed for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect WRITE speed for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_WRITE messages" | sed -r "s/BULK_WRITE messages: //g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkWriteMsg=$(bc <<< "$blkWriteMsg + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] BULK_WRITE messages: $cnt"
+ else
+ echo "[WARN] BULK_WRITE messages count is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "BULK_WRITE messages count is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_WRITE messages count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_WRITE messages count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_WRITE errors" | sed -r "s/BULK_WRITE errors: //g" | sed -r "s/,.*//g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkWriteErrors=$(bc <<< "$blkWriteErrors + $cnt")
+ echo "[INFO] BULK_WRITE errors: $cnt"
+ if [ $cnt -ne 0 ]; then
+ if [ -n "$blkWriteErrNodes" ]; then
+ blkWriteErrNodes="${blkWriteErrNodes}, "
+ fi
+ blkWriteErrNodes="${blkWriteErrNodes}${node}"
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_WRITE errors count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_WRITE errors count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_WRITE speed" | sed -r "s/BULK_WRITE speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkWriteSpeed=$(bc <<< "$blkWriteSpeed + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] BULK_WRITE speed: $cnt msg/sec"
+ else
+ echo "[WARN] BULK_WRITE speed is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "BULK_WRITE speed is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_WRITE speed for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_WRITE speed for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^READ messages" | sed -r "s/READ messages: //g" | xargs)
+ if [ -n "$cnt" ]; then
+ readMsg=$(bc <<< "$readMsg + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] READ messages: $cnt"
+ else
+ echo "[WARN] READ messages count is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "READ messages count is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect READ messages count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect READ messages count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^READ errors" | sed -r "s/READ errors: //g" | sed -r "s/,.*//g" | xargs)
+ if [ -n "$cnt" ]; then
+ readErrors=$(bc <<< "$readErrors + $cnt")
+ echo "[INFO] READ errors: $cnt"
+ if [ $cnt -ne 0 ]; then
+ if [ -n "$readErrNodes" ]; then
+ blkWriteErrNodes="${readErrNodes}, "
+ fi
+ readErrNodes="${readErrNodes}${node}"
+ fi
+ else
+ echo "[WARN] Failed to detect READ errors count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect READ errors count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^READ speed" | sed -r "s/READ speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+ if [ -n "$cnt" ]; then
+ readSpeed=$(bc <<< "$readSpeed + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] READ speed: $cnt msg/sec"
+ else
+ echo "[WARN] READ speed is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "READ speed is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect READ speed for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect READ speed for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_READ messages" | sed -r "s/BULK_READ messages: //g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkReadMsg=$(bc <<< "$blkReadMsg + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] BULK_READ messages: $cnt"
+ else
+ echo "[WARN] BULK_READ messages count is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "BULK_READ messages count is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_READ messages count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_READ messages count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_READ errors" | sed -r "s/BULK_READ errors: //g" | sed -r "s/,.*//g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkReadErrors=$(bc <<< "$blkReadErrors + $cnt")
+ echo "[INFO] BULK_READ errors: $cnt"
+ if [ $cnt -ne 0 ]; then
+ if [ -n "$blkReadErrNodes" ]; then
+ blkReadErrNodes="${blkReadErrNodes}, "
+ fi
+ blkReadErrNodes="${blkReadErrNodes}${node}"
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_READ errors count for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_READ errors count for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+
+ cnt=$(cat $logFile | grep "^BULK_READ speed" | sed -r "s/BULK_READ speed: //g" | sed -r "s/ msg\/sec//g" | xargs)
+ if [ -n "$cnt" ]; then
+ blkReadSpeed=$(bc <<< "$blkReadSpeed + $cnt")
+ if [ $cnt -ne 0 ]; then
+ echo "[INFO] BULK_READ speed: $cnt msg/sec"
+ else
+ echo "[WARN] BULK_READ speed is zero for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "BULK_READ speed is zero for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ else
+ echo "[WARN] Failed to detect BULK_READ speed for $node node. This test probably failed."
+ echo "WARNING |" >> $tmpFile
+ echo "--------" >> $tmpFile
+ echo "Failed to detect BULK_READ speed for $node node. This test probably failed." >> $tmpFile
+ echo "----------------------------------------------------------------------------------------------" >> $tmpFile
+ fi
+ done
+
+ echo "-------------------------------------------------------------------------------------"
+
+ echo "WRITE test metrics |" >> $1
+ echo "-------------------" >> $1
+ echo "Messages: $writeMsg" >> $1
+ echo "Speed : $writeSpeed msg/sec" >> $1
+ echo "Errors : $writeErrors" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+
+ echo "BULK_WRITE test metrics |" >> $1
+ echo "------------------------" >> $1
+ echo "Messages: $blkWriteMsg" >> $1
+ echo "Speed : $blkWriteSpeed msg/sec" >> $1
+ echo "Errors : $blkWriteErrors" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+
+ echo "READ test metrics |" >> $1
+ echo "------------------" >> $1
+ echo "Messages: $readMsg" >> $1
+ echo "Speed : $readSpeed msg/sec" >> $1
+ echo "Errors : $readErrors" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+
+ echo "BULK_READ test metrics |" >> $1
+ echo "-----------------------" >> $1
+ echo "Messages: $blkReadMsg" >> $1
+ echo "Speed : $blkReadSpeed msg/sec" >> $1
+ echo "Errors : $blkReadErrors" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+
+ if [ -n "$writeErrNodes" ]; then
+ echo "Nodes having WRITE errors |" >> $1
+ echo "-------------------------------" >> $1
+ echo "$writeErrNodes" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+ fi
+
+ if [ -n "$blkWriteErrNodes" ]; then
+ echo "Nodes having BULK_WRITE errors |" >> $1
+ echo "-------------------------------" >> $1
+ echo "$blkWriteErrNodes" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+ fi
+
+ if [ -n "$readErrNodes" ]; then
+ echo "Nodes having READ errors |" >> $1
+ echo "-------------------------------" >> $1
+ echo "$readErrNodes" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+ fi
+
+ if [ -n "$blkReadErrNodes" ]; then
+ echo "Nodes having BULK_READ errors |" >> $1
+ echo "-------------------------------" >> $1
+ echo "$blkReadErrNodes" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+ fi
+
+ cat $tmpFile >> $1
+
+ rm -f $tmpFile
+}
+
+# Creates report for failed tests
+reportFailedTestsDetailes()
+{
+ for dir in $2/*
+ do
+ node=$(echo $dir | sed -r "s/^.*\///g")
+ if [ -z "$node" ]; then
+ continue
+ fi
+
+ echo "----------------------------------------------------------------------------------------------" >> $1
+ echo "Error details for node: $node" >> $1
+ echo "----------------------------------------------------------------------------------------------" >> $1
+
+ if [ -f "$dir/__error__" ]; then
+ cat $dir/__error__ >> $1
+ else
+ echo "N/A" >> $1
+ fi
+ done
+}
+
+#######################################################################################################
+
+if [ "$TESTS_TYPE" != "ignite" ] && [ "$TESTS_TYPE" != "cassandra" ]; then
+ terminate "Incorrect tests type specified: $TESTS_TYPE"
+fi
+
+reportTestsSummary
\ No newline at end of file
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraConfigTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraConfigTest.java
new file mode 100644
index 000000000..48ac18050
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraConfigTest.java
@@ -0,0 +1,141 @@
+/*
+ * 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.ignite.tests;
+
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyPersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Simple test for DDL generator.
+ */
+public class CassandraConfigTest {
+ /**
+ * Check if same DDL generated for similar keys and same KeyPersistenceConfiguration.
+ */
+ @Test
+ public void testDDLGeneration() {
+ KeyPersistenceSettings keyPersistenceSettingsA = getKeyPersistenceSettings(KeyA.class);
+ KeyPersistenceSettings keyPersistenceSettingsB = getKeyPersistenceSettings(KeyB.class);
+
+ assertEquals(keyPersistenceSettingsB.getPrimaryKeyDDL(),
+ keyPersistenceSettingsA.getPrimaryKeyDDL());
+
+ assertEquals(keyPersistenceSettingsB.getClusteringDDL(),
+ keyPersistenceSettingsA.getClusteringDDL());
+ }
+
+ /**
+ * @return KeyPersistenceSetting
+ */
+ private KeyPersistenceSettings getKeyPersistenceSettings(Class keyClass) {
+ String cfg = "" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " \n" +
+ " " +
+ " " +
+ " " +
+ " ";
+
+ return new KeyValuePersistenceSettings(cfg).getKeyPersistenceSettings();
+ }
+
+ /**
+ *
+ */
+ public static class BaseKey {
+ /** */
+ @QuerySqlField
+ // Looks like next annotation is ignored when generating DDL,
+ // but Ignite supports this annotation in parent classes.
+// @AffinityKeyMapped
+ private Integer contextId;
+
+ /** */
+ public Integer getContextId() {
+ return contextId;
+ }
+
+ /** */
+ public void setContextId(Integer contextId) {
+ this.contextId = contextId;
+ }
+ }
+
+ /**
+ *
+ */
+ public static class KeyA extends BaseKey {
+ /** */
+ @QuerySqlField(index = true)
+ private String timestamp;
+
+ /** */
+ @QuerySqlField(index = true)
+ private String name;
+
+ /** */
+ @QuerySqlField
+ private String creationDate;
+
+ /**
+ * Constructor.
+ */
+ public KeyA() {
+ }
+ }
+
+ /**
+ *
+ */
+ public static class KeyB {
+
+ /** */
+ @QuerySqlField(index = true)
+ private String timestamp;
+
+ /** */
+ @QuerySqlField(index = true)
+ private String name;
+
+ /** */
+ @QuerySqlField
+ private String creationDate;
+
+ /** */
+ @QuerySqlField
+// @AffinityKeyMapped
+ private Integer contextId;
+
+ /**
+ * Constructor.
+ */
+ public KeyB() {
+ }
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java
new file mode 100644
index 000000000..48f85c342
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java
@@ -0,0 +1,108 @@
+/*
+ * 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.ignite.tests;
+
+import org.apache.ignite.tests.load.LoadTestDriver;
+import org.apache.ignite.tests.load.cassandra.BulkReadWorker;
+import org.apache.ignite.tests.load.cassandra.BulkWriteWorker;
+import org.apache.ignite.tests.load.cassandra.ReadWorker;
+import org.apache.ignite.tests.load.cassandra.WriteWorker;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * Load tests for {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore} implementation of
+ * {@link org.apache.ignite.cache.store.CacheStore} which allows to store Ignite cache data into Cassandra tables.
+ */
+public class CassandraDirectPersistenceLoadTest extends LoadTestDriver {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger("CassandraLoadTests");
+
+ /**
+ *
+ * @param args Test arguments.
+ */
+ public static void main(String[] args) {
+ try {
+ LOGGER.info("Cassandra load tests execution started");
+
+ LoadTestDriver driver = new CassandraDirectPersistenceLoadTest();
+
+ /**
+ * Load test scripts could be executed from several machines. Current implementation can correctly,
+ * handle situation when Cassandra keyspace/table was dropped - for example by the same load test
+ * started a bit later on another machine. Moreover there is a warm up period for each load test.
+ * Thus all the delays related to keyspaces/tables recreation actions will not affect performance metrics,
+ * but it will be produced lots of "trash" output in the logs (related to correct handling of such
+ * exceptional situation and keyspace/table recreation).
+ *
+ * Thus dropping test keyspaces at the beginning of the tests makes sense only for Unit tests,
+ * but not for Load tests.
+ **/
+
+ //CassandraHelper.dropTestKeyspaces();
+
+ driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME);
+
+ driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME);
+
+ driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME);
+
+ driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME);
+
+ /**
+ * Load test script executed on one machine could complete earlier that the same load test executed from
+ * another machine. Current implementation can correctly handle situation when Cassandra keyspace/table
+ * was dropped (simply recreate it). But dropping keyspace/table during load tests execution and subsequent
+ * recreation of such objects can have SIGNIFICANT EFFECT on final performance metrics.
+ *
+ * Thus dropping test keyspaces at the end of the tests makes sense only for Unit tests,
+ * but not for Load tests.
+ */
+
+ //CassandraHelper.dropTestKeyspaces(); // REVIEW This line is commented by purpose?
+
+ LOGGER.info("Cassandra load tests execution completed");
+ }
+ catch (Throwable e) {
+ LOGGER.error("Cassandra load tests execution failed", e);
+ throw new RuntimeException("Cassandra load tests execution failed", e);
+ }
+ finally {
+ CassandraHelper.releaseCassandraResources();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Logger logger() {
+ return LOGGER;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Object setup(String logName) {
+ return CacheStoreHelper.createCacheStore(
+ TestsHelper.getLoadTestsCacheName(),
+ TestsHelper.getLoadTestsPersistenceSettings(),
+ CassandraHelper.getAdminDataSrc(),
+ LogManager.getLogger(logName));
+ }
+
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java
new file mode 100644
index 000000000..0bc33677b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java
@@ -0,0 +1,767 @@
+/*
+ * 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.ignite.tests;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.tests.pojos.Person;
+import org.apache.ignite.tests.pojos.PersonId;
+import org.apache.ignite.tests.pojos.Product;
+import org.apache.ignite.tests.pojos.ProductOrder;
+import org.apache.ignite.tests.pojos.SimplePerson;
+import org.apache.ignite.tests.pojos.SimplePersonId;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestCacheSession;
+import org.apache.ignite.tests.utils.TestTransaction;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.core.io.ClassPathResource;
+
+/**
+ * Unit tests for {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore} implementation of
+ * {@link org.apache.ignite.cache.store.CacheStore} which allows to store Ignite cache data into Cassandra tables.
+ */
+public class CassandraDirectPersistenceTest {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger(CassandraDirectPersistenceTest.class.getName());
+
+ /** */
+ @BeforeClass
+ public static void setUpClass() {
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.startEmbeddedCassandra(LOGGER);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+ }
+ }
+
+ LOGGER.info("Testing admin connection to Cassandra");
+ CassandraHelper.testAdminConnection();
+
+ LOGGER.info("Testing regular connection to Cassandra");
+ CassandraHelper.testRegularConnection();
+
+ LOGGER.info("Dropping all artifacts from previous tests execution session");
+ CassandraHelper.dropTestKeyspaces();
+
+ LOGGER.info("Start tests execution");
+ }
+
+ /** */
+ @AfterClass
+ public static void tearDownClass() {
+ try {
+ CassandraHelper.dropTestKeyspaces();
+ }
+ finally {
+ CassandraHelper.releaseCassandraResources();
+
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.stopEmbeddedCassandra();
+ }
+ catch (Throwable e) {
+ LOGGER.error("Failed to stop embedded Cassandra instance", e);
+ }
+ }
+ }
+ }
+
+ /** */
+ @Test
+ @SuppressWarnings("unchecked")
+ public void primitiveStrategyTest() {
+ CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store2 = CacheStoreHelper.createCacheStore("stringTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ Collection> longEntries = TestsHelper.generateLongsEntries();
+ Collection> strEntries = TestsHelper.generateStringsEntries();
+
+ Collection fakeLongKeys = TestsHelper.getKeys(longEntries);
+ fakeLongKeys.add(-1L);
+ fakeLongKeys.add(-2L);
+ fakeLongKeys.add(-3L);
+ fakeLongKeys.add(-4L);
+
+ Collection fakeStrKeys = TestsHelper.getKeys(strEntries);
+ fakeStrKeys.add("-1");
+ fakeStrKeys.add("-2");
+ fakeStrKeys.add("-3");
+ fakeStrKeys.add("-4");
+
+ LOGGER.info("Running PRIMITIVE strategy write tests");
+
+ LOGGER.info("Running single write operation tests");
+ store1.write(longEntries.iterator().next());
+ store2.write(strEntries.iterator().next());
+ LOGGER.info("Single write operation tests passed");
+
+ LOGGER.info("Running bulk write operation tests");
+ store1.writeAll(longEntries);
+ store2.writeAll(strEntries);
+ LOGGER.info("Bulk write operation tests passed");
+
+ LOGGER.info("PRIMITIVE strategy write tests passed");
+
+ LOGGER.info("Running PRIMITIVE strategy read tests");
+
+ LOGGER.info("Running single read operation tests");
+
+ LOGGER.info("Running real keys read tests");
+
+ Long longVal = (Long)store1.load(longEntries.iterator().next().getKey());
+ if (!longEntries.iterator().next().getValue().equals(longVal))
+ throw new RuntimeException("Long values were incorrectly deserialized from Cassandra");
+
+ String strVal = (String)store2.load(strEntries.iterator().next().getKey());
+ if (!strEntries.iterator().next().getValue().equals(strVal))
+ throw new RuntimeException("String values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Running fake keys read tests");
+
+ longVal = (Long)store1.load(-1L);
+ if (longVal != null)
+ throw new RuntimeException("Long value with fake key '-1' was found in Cassandra");
+
+ strVal = (String)store2.load("-1");
+ if (strVal != null)
+ throw new RuntimeException("String value with fake key '-1' was found in Cassandra");
+
+ LOGGER.info("Single read operation tests passed");
+
+ LOGGER.info("Running bulk read operation tests");
+
+ LOGGER.info("Running real keys read tests");
+
+ Map longValues = store1.loadAll(TestsHelper.getKeys(longEntries));
+ if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+ throw new RuntimeException("Long values were incorrectly deserialized from Cassandra");
+
+ Map strValues = store2.loadAll(TestsHelper.getKeys(strEntries));
+ if (!TestsHelper.checkCollectionsEqual(strValues, strEntries))
+ throw new RuntimeException("String values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Running fake keys read tests");
+
+ longValues = store1.loadAll(fakeLongKeys);
+ if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+ throw new RuntimeException("Long values were incorrectly deserialized from Cassandra");
+
+ strValues = store2.loadAll(fakeStrKeys);
+ if (!TestsHelper.checkCollectionsEqual(strValues, strEntries))
+ throw new RuntimeException("String values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk read operation tests passed");
+
+ LOGGER.info("PRIMITIVE strategy read tests passed");
+
+ LOGGER.info("Running PRIMITIVE strategy delete tests");
+
+ LOGGER.info("Deleting real keys");
+
+ store1.delete(longEntries.iterator().next().getKey());
+ store1.deleteAll(TestsHelper.getKeys(longEntries));
+
+ store2.delete(strEntries.iterator().next().getKey());
+ store2.deleteAll(TestsHelper.getKeys(strEntries));
+
+ LOGGER.info("Deleting fake keys");
+
+ store1.delete(-1L);
+ store2.delete("-1");
+
+ store1.deleteAll(fakeLongKeys);
+ store2.deleteAll(fakeStrKeys);
+
+ LOGGER.info("PRIMITIVE strategy delete tests passed");
+ }
+
+ /** */
+ @Test
+ @SuppressWarnings("unchecked")
+ public void blobStrategyTest() {
+ CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ Collection> longEntries = TestsHelper.generateLongsEntries();
+ Collection> personEntries = TestsHelper.generateLongsPersonsEntries();
+
+ LOGGER.info("Running BLOB strategy write tests");
+
+ LOGGER.info("Running single write operation tests");
+ store1.write(longEntries.iterator().next());
+ store2.write(personEntries.iterator().next());
+ store3.write(personEntries.iterator().next());
+ LOGGER.info("Single write operation tests passed");
+
+ LOGGER.info("Running bulk write operation tests");
+ store1.writeAll(longEntries);
+ store2.writeAll(personEntries);
+ store3.writeAll(personEntries);
+ LOGGER.info("Bulk write operation tests passed");
+
+ LOGGER.info("BLOB strategy write tests passed");
+
+ LOGGER.info("Running BLOB strategy read tests");
+
+ LOGGER.info("Running single read operation tests");
+
+ Long longVal = (Long)store1.load(longEntries.iterator().next().getKey());
+ if (!longEntries.iterator().next().getValue().equals(longVal))
+ throw new RuntimeException("Long values were incorrectly deserialized from Cassandra");
+
+ Person personVal = (Person)store2.load(personEntries.iterator().next().getKey());
+ if (!personEntries.iterator().next().getValue().equals(personVal))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ personVal = (Person)store3.load(personEntries.iterator().next().getKey());
+ if (!personEntries.iterator().next().getValue().equals(personVal))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Single read operation tests passed");
+
+ LOGGER.info("Running bulk read operation tests");
+
+ Map longValues = store1.loadAll(TestsHelper.getKeys(longEntries));
+ if (!TestsHelper.checkCollectionsEqual(longValues, longEntries))
+ throw new RuntimeException("Long values were incorrectly deserialized from Cassandra");
+
+ Map personValues = store2.loadAll(TestsHelper.getKeys(personEntries));
+ if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ personValues = store3.loadAll(TestsHelper.getKeys(personEntries));
+ if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk read operation tests passed");
+
+ LOGGER.info("BLOB strategy read tests passed");
+
+ LOGGER.info("Running BLOB strategy delete tests");
+
+ store1.delete(longEntries.iterator().next().getKey());
+ store1.deleteAll(TestsHelper.getKeys(longEntries));
+
+ store2.delete(personEntries.iterator().next().getKey());
+ store2.deleteAll(TestsHelper.getKeys(personEntries));
+
+ store3.delete(personEntries.iterator().next().getKey());
+ store3.deleteAll(TestsHelper.getKeys(personEntries));
+
+ LOGGER.info("BLOB strategy delete tests passed");
+ }
+
+ /** */
+ @Test
+ @SuppressWarnings("unchecked")
+ public void pojoStrategyTest() {
+ CacheStore store1 = CacheStoreHelper.createCacheStore("longTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store4 = CacheStoreHelper.createCacheStore("persons",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-4.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore productStore = CacheStoreHelper.createCacheStore("product",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/product.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore orderStore = CacheStoreHelper.createCacheStore("order",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/order.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ Collection> entries1 = TestsHelper.generateLongsPersonsEntries();
+ Collection> entries2 = TestsHelper.generatePersonIdsPersonsEntries();
+ Collection> entries3 = TestsHelper.generatePersonIdsPersonsEntries();
+ Collection> productEntries = TestsHelper.generateProductEntries();
+ Collection> orderEntries = TestsHelper.generateOrderEntries();
+
+ LOGGER.info("Running POJO strategy write tests");
+
+ LOGGER.info("Running single write operation tests");
+ store1.write(entries1.iterator().next());
+ store2.write(entries2.iterator().next());
+ store3.write(entries3.iterator().next());
+ store4.write(entries3.iterator().next());
+ productStore.write(productEntries.iterator().next());
+ orderStore.write(orderEntries.iterator().next());
+ LOGGER.info("Single write operation tests passed");
+
+ LOGGER.info("Running bulk write operation tests");
+ store1.writeAll(entries1);
+ store2.writeAll(entries2);
+ store3.writeAll(entries3);
+ store4.writeAll(entries3);
+ productStore.writeAll(productEntries);
+ orderStore.writeAll(orderEntries);
+ LOGGER.info("Bulk write operation tests passed");
+
+ LOGGER.info("POJO strategy write tests passed");
+
+ LOGGER.info("Running POJO strategy read tests");
+
+ LOGGER.info("Running single read operation tests");
+
+ Person person = (Person)store1.load(entries1.iterator().next().getKey());
+ if (!entries1.iterator().next().getValue().equalsPrimitiveFields(person))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ person = (Person)store2.load(entries2.iterator().next().getKey());
+ if (!entries2.iterator().next().getValue().equalsPrimitiveFields(person))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ person = (Person)store3.load(entries3.iterator().next().getKey());
+ if (!entries3.iterator().next().getValue().equals(person))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ person = (Person)store4.load(entries3.iterator().next().getKey());
+ if (!entries3.iterator().next().getValue().equals(person))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ Product product = (Product)productStore.load(productEntries.iterator().next().getKey());
+ if (!productEntries.iterator().next().getValue().equals(product))
+ throw new RuntimeException("Product values were incorrectly deserialized from Cassandra");
+
+ ProductOrder order = (ProductOrder)orderStore.load(orderEntries.iterator().next().getKey());
+ if (!orderEntries.iterator().next().getValue().equals(order))
+ throw new RuntimeException("Order values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Single read operation tests passed");
+
+ LOGGER.info("Running bulk read operation tests");
+
+ Map persons = store1.loadAll(TestsHelper.getKeys(entries1));
+ if (!TestsHelper.checkPersonCollectionsEqual(persons, entries1, true))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ persons = store2.loadAll(TestsHelper.getKeys(entries2));
+ if (!TestsHelper.checkPersonCollectionsEqual(persons, entries2, true))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ persons = store3.loadAll(TestsHelper.getKeys(entries3));
+ if (!TestsHelper.checkPersonCollectionsEqual(persons, entries3, false))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ persons = store4.loadAll(TestsHelper.getKeys(entries3));
+ if (!TestsHelper.checkPersonCollectionsEqual(persons, entries3, false))
+ throw new RuntimeException("Person values were incorrectly deserialized from Cassandra");
+
+ Map products = productStore.loadAll(TestsHelper.getKeys(productEntries));
+ if (!TestsHelper.checkProductCollectionsEqual(products, productEntries))
+ throw new RuntimeException("Product values were incorrectly deserialized from Cassandra");
+
+ Map orders = orderStore.loadAll(TestsHelper.getKeys(orderEntries));
+ if (!TestsHelper.checkOrderCollectionsEqual(orders, orderEntries))
+ throw new RuntimeException("Order values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk read operation tests passed");
+
+ LOGGER.info("POJO strategy read tests passed");
+
+ LOGGER.info("Running POJO strategy delete tests");
+
+ store1.delete(entries1.iterator().next().getKey());
+ store1.deleteAll(TestsHelper.getKeys(entries1));
+
+ store2.delete(entries2.iterator().next().getKey());
+ store2.deleteAll(TestsHelper.getKeys(entries2));
+
+ store3.delete(entries3.iterator().next().getKey());
+ store3.deleteAll(TestsHelper.getKeys(entries3));
+
+ store4.delete(entries3.iterator().next().getKey());
+ store4.deleteAll(TestsHelper.getKeys(entries3));
+
+ productStore.delete(productEntries.iterator().next().getKey());
+ productStore.deleteAll(TestsHelper.getKeys(productEntries));
+
+ orderStore.delete(orderEntries.iterator().next().getKey());
+ orderStore.deleteAll(TestsHelper.getKeys(orderEntries));
+
+ LOGGER.info("POJO strategy delete tests passed");
+ }
+
+ /** */
+ @Test
+ @SuppressWarnings("unchecked")
+ public void pojoStrategySimpleObjectsTest() {
+ CacheStore store5 = CacheStoreHelper.createCacheStore("persons5",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-5.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ CacheStore store6 = CacheStoreHelper.createCacheStore("persons6",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-6.xml"),
+ CassandraHelper.getAdminDataSrc());
+
+ Collection> entries5 = TestsHelper.generateSimplePersonIdsPersonsEntries();
+ Collection> entries6 = TestsHelper.generateSimplePersonIdsPersonsEntries();
+
+ LOGGER.info("Running POJO strategy write tests for simple objects");
+
+ LOGGER.info("Running single write operation tests");
+ store5.write(entries5.iterator().next());
+ store6.write(entries6.iterator().next());
+ LOGGER.info("Single write operation tests passed");
+
+ LOGGER.info("Running bulk write operation tests");
+ store5.writeAll(entries5);
+ store6.writeAll(entries6);
+ LOGGER.info("Bulk write operation tests passed");
+
+ LOGGER.info("POJO strategy write tests for simple objects passed");
+
+ LOGGER.info("Running POJO simple objects strategy read tests");
+
+ LOGGER.info("Running single read operation tests");
+
+ SimplePerson person = (SimplePerson)store5.load(entries5.iterator().next().getKey());
+ if (!entries5.iterator().next().getValue().equalsPrimitiveFields(person))
+ throw new RuntimeException("SimplePerson values were incorrectly deserialized from Cassandra");
+
+ person = (SimplePerson)store6.load(entries6.iterator().next().getKey());
+ if (!entries6.iterator().next().getValue().equalsPrimitiveFields(person))
+ throw new RuntimeException("SimplePerson values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Single read operation tests passed");
+
+ LOGGER.info("Running bulk read operation tests");
+
+ Map persons = store5.loadAll(TestsHelper.getKeys(entries5));
+ if (!TestsHelper.checkSimplePersonCollectionsEqual(persons, entries5, true))
+ throw new RuntimeException("SimplePerson values were incorrectly deserialized from Cassandra");
+
+ persons = store6.loadAll(TestsHelper.getKeys(entries6));
+ if (!TestsHelper.checkSimplePersonCollectionsEqual(persons, entries6, true))
+ throw new RuntimeException("SimplePerson values were incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk read operation tests passed");
+
+ LOGGER.info("POJO strategy read tests for simple objects passed");
+
+ LOGGER.info("Running POJO strategy delete tests for simple objects");
+
+ store5.delete(entries5.iterator().next().getKey());
+ store5.deleteAll(TestsHelper.getKeys(entries5));
+
+ store6.delete(entries6.iterator().next().getKey());
+ store6.deleteAll(TestsHelper.getKeys(entries6));
+
+ LOGGER.info("POJO strategy delete tests for simple objects passed");
+ }
+
+ /** */
+ @Test
+ @SuppressWarnings("unchecked")
+ public void pojoStrategyTransactionTest() {
+ Map sessionProps = U.newHashMap(1);
+ Transaction sessionTx = new TestTransaction();
+
+ CacheStore productStore = CacheStoreHelper.createCacheStore("product",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/product.xml"),
+ CassandraHelper.getAdminDataSrc(), new TestCacheSession("product", sessionTx, sessionProps));
+
+ CacheStore orderStore = CacheStoreHelper.createCacheStore("order",
+ new ClassPathResource("org/apache/ignite/tests/persistence/pojo/order.xml"),
+ CassandraHelper.getAdminDataSrc(), new TestCacheSession("order", sessionTx, sessionProps));
+
+ List> productEntries = TestsHelper.generateProductEntries();
+ Map>> ordersPerProduct =
+ TestsHelper.generateOrdersPerProductEntries(productEntries, 2);
+
+ Collection productIds = TestsHelper.getProductIds(productEntries);
+ Collection orderIds = TestsHelper.getOrderIds(ordersPerProduct);
+
+ LOGGER.info("Running POJO strategy transaction write tests");
+
+ LOGGER.info("Running single write operation tests");
+
+ CassandraHelper.dropTestKeyspaces();
+
+ Product product = productEntries.iterator().next().getValue();
+ ProductOrder order = ordersPerProduct.get(product.getId()).iterator().next().getValue();
+
+ productStore.write(productEntries.iterator().next());
+ orderStore.write(ordersPerProduct.get(product.getId()).iterator().next());
+
+ if (productStore.load(product.getId()) != null || orderStore.load(order.getId()) != null) {
+ throw new RuntimeException("Single write operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already persisted into Cassandra");
+ }
+
+ Map products = (Map)productStore.loadAll(productIds);
+ Map orders = (Map)orderStore.loadAll(orderIds);
+
+ if ((products != null && !products.isEmpty()) || (orders != null && !orders.isEmpty())) {
+ throw new RuntimeException("Single write operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already persisted into Cassandra");
+ }
+
+ //noinspection deprecation
+ orderStore.sessionEnd(true);
+ //noinspection deprecation
+ productStore.sessionEnd(true);
+
+ Product product1 = (Product)productStore.load(product.getId());
+ ProductOrder order1 = (ProductOrder)orderStore.load(order.getId());
+
+ if (product1 == null || order1 == null) {
+ throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+ "no objects were persisted into Cassandra");
+ }
+
+ if (!product.equals(product1) || !order.equals(order1)) {
+ throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+ "objects were incorrectly persisted/loaded to/from Cassandra");
+ }
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if (products == null || products.isEmpty() || orders == null || orders.isEmpty()) {
+ throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+ "no objects were persisted into Cassandra");
+ }
+
+ if (products.size() > 1 || orders.size() > 1) {
+ throw new RuntimeException("Single write operation test failed. There were committed more objects " +
+ "into Cassandra than expected");
+ }
+
+ product1 = products.entrySet().iterator().next().getValue();
+ order1 = orders.entrySet().iterator().next().getValue();
+
+ if (!product.equals(product1) || !order.equals(order1)) {
+ throw new RuntimeException("Single write operation test failed. Transaction was committed, but " +
+ "objects were incorrectly persisted/loaded to/from Cassandra");
+ }
+
+ LOGGER.info("Single write operation tests passed");
+
+ LOGGER.info("Running bulk write operation tests");
+
+ CassandraHelper.dropTestKeyspaces();
+ sessionProps.clear();
+
+ productStore.writeAll(productEntries);
+
+ for (Long productId : ordersPerProduct.keySet())
+ orderStore.writeAll(ordersPerProduct.get(productId));
+
+ for (Long productId : productIds) {
+ if (productStore.load(productId) != null) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already persisted into Cassandra");
+ }
+ }
+
+ for (Long orderId : orderIds) {
+ if (orderStore.load(orderId) != null) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already persisted into Cassandra");
+ }
+ }
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if ((products != null && !products.isEmpty()) || (orders != null && !orders.isEmpty())) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already persisted into Cassandra");
+ }
+
+ //noinspection deprecation
+ productStore.sessionEnd(true);
+ //noinspection deprecation
+ orderStore.sessionEnd(true);
+
+ for (CacheEntryImpl entry : productEntries) {
+ product = (Product)productStore.load(entry.getKey());
+
+ if (!entry.getValue().equals(product)) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+ "not all objects were persisted into Cassandra");
+ }
+ }
+
+ for (Long productId : ordersPerProduct.keySet()) {
+ for (CacheEntryImpl entry : ordersPerProduct.get(productId)) {
+ order = (ProductOrder)orderStore.load(entry.getKey());
+
+ if (!entry.getValue().equals(order)) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+ "not all objects were persisted into Cassandra");
+ }
+ }
+ }
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if (products == null || products.isEmpty() || orders == null || orders.isEmpty()) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+ "no objects were persisted into Cassandra");
+ }
+
+ if (products.size() < productIds.size() || orders.size() < orderIds.size()) {
+ throw new RuntimeException("Bulk write operation test failed. There were committed less objects " +
+ "into Cassandra than expected");
+ }
+
+ if (products.size() > productIds.size() || orders.size() > orderIds.size()) {
+ throw new RuntimeException("Bulk write operation test failed. There were committed more objects " +
+ "into Cassandra than expected");
+ }
+
+ for (CacheEntryImpl entry : productEntries) {
+ product = products.get(entry.getKey());
+
+ if (!entry.getValue().equals(product)) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+ "some objects were incorrectly persisted/loaded to/from Cassandra");
+ }
+ }
+
+ for (Long productId : ordersPerProduct.keySet()) {
+ for (CacheEntryImpl entry : ordersPerProduct.get(productId)) {
+ order = orders.get(entry.getKey());
+
+ if (!entry.getValue().equals(order)) {
+ throw new RuntimeException("Bulk write operation test failed. Transaction was committed, but " +
+ "some objects were incorrectly persisted/loaded to/from Cassandra");
+ }
+ }
+ }
+
+ LOGGER.info("Bulk write operation tests passed");
+
+ LOGGER.info("POJO strategy transaction write tests passed");
+
+ LOGGER.info("Running POJO strategy transaction delete tests");
+
+ LOGGER.info("Running single delete tests");
+
+ sessionProps.clear();
+
+ Product deletedProduct = productEntries.remove(0).getValue();
+ ProductOrder deletedOrder = ordersPerProduct.get(deletedProduct.getId()).remove(0).getValue();
+
+ productStore.delete(deletedProduct.getId());
+ orderStore.delete(deletedOrder.getId());
+
+ if (productStore.load(deletedProduct.getId()) == null || orderStore.load(deletedOrder.getId()) == null) {
+ throw new RuntimeException("Single delete operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already deleted from Cassandra");
+ }
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if (products.size() != productIds.size() || orders.size() != orderIds.size()) {
+ throw new RuntimeException("Single delete operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already deleted from Cassandra");
+ }
+
+ //noinspection deprecation
+ productStore.sessionEnd(true);
+ //noinspection deprecation
+ orderStore.sessionEnd(true);
+
+ if (productStore.load(deletedProduct.getId()) != null || orderStore.load(deletedOrder.getId()) != null) {
+ throw new RuntimeException("Single delete operation test failed. Transaction was committed, but " +
+ "objects were not deleted from Cassandra");
+ }
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if (products.get(deletedProduct.getId()) != null || orders.get(deletedOrder.getId()) != null) {
+ throw new RuntimeException("Single delete operation test failed. Transaction was committed, but " +
+ "objects were not deleted from Cassandra");
+ }
+
+ LOGGER.info("Single delete tests passed");
+
+ LOGGER.info("Running bulk delete tests");
+
+ sessionProps.clear();
+
+ productStore.deleteAll(productIds);
+ orderStore.deleteAll(orderIds);
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if (products == null || products.isEmpty() || orders == null || orders.isEmpty()) {
+ throw new RuntimeException("Bulk delete operation test failed. Transaction wasn't committed yet, but " +
+ "objects were already deleted from Cassandra");
+ }
+
+ //noinspection deprecation
+ orderStore.sessionEnd(true);
+ //noinspection deprecation
+ productStore.sessionEnd(true);
+
+ products = (Map)productStore.loadAll(productIds);
+ orders = (Map)orderStore.loadAll(orderIds);
+
+ if ((products != null && !products.isEmpty()) || (orders != null && !orders.isEmpty())) {
+ throw new RuntimeException("Bulk delete operation test failed. Transaction was committed, but " +
+ "objects were not deleted from Cassandra");
+ }
+
+ LOGGER.info("Bulk delete tests passed");
+
+ LOGGER.info("POJO strategy transaction delete tests passed");
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraLocalServer.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraLocalServer.java
new file mode 100644
index 000000000..a229d955e
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraLocalServer.java
@@ -0,0 +1,60 @@
+/*
+ * 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.ignite.tests;
+
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * Simple helper class to run Cassandra on localhost
+ */
+public class CassandraLocalServer {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger(CassandraLocalServer.class.getName());
+
+ /** */
+ public static void main(String[] args) {
+ try {
+ CassandraHelper.startEmbeddedCassandra(LOGGER);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+ }
+
+ LOGGER.info("Testing admin connection to Cassandra");
+ CassandraHelper.testAdminConnection();
+
+ LOGGER.info("Testing regular connection to Cassandra");
+ CassandraHelper.testRegularConnection();
+
+ LOGGER.info("Dropping all artifacts from previous tests execution session");
+ CassandraHelper.dropTestKeyspaces();
+
+ while (true) {
+ try {
+ System.out.println("Cassandra server running");
+
+ Thread.sleep(10000);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Cassandra server terminated", e);
+ }
+ }
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraSessionImplTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraSessionImplTest.java
new file mode 100644
index 000000000..a3a2bcdb7
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/CassandraSessionImplTest.java
@@ -0,0 +1,211 @@
+/*
+ * 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.ignite.tests;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import com.datastax.driver.core.BoundStatement;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ColumnDefinitions;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.PreparedId;
+import com.datastax.driver.core.PreparedStatement;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.ResultSetFuture;
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.Session;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.cache.store.cassandra.session.BatchExecutionAssistant;
+import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
+import org.apache.ignite.cache.store.cassandra.session.WrappedPreparedStatement;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.nullable;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/** */
+public class CassandraSessionImplTest {
+
+ /** */
+ private PreparedStatement preparedStatement1 = mockPreparedStatement();
+
+ /** */
+ private PreparedStatement preparedStatement2 = mockPreparedStatement();
+
+ /** */
+ private MyBoundStatement1 boundStatement1 = new MyBoundStatement1(preparedStatement1);
+
+ /** */
+ private MyBoundStatement2 boundStatement2 = new MyBoundStatement2(preparedStatement2);
+
+ /** */
+ @SuppressWarnings("unchecked")
+ @Test
+ public void executeFailureTest() {
+ Session session1 = mock(Session.class);
+ Session session2 = mock(Session.class);
+ when(session1.prepare(nullable(String.class))).thenReturn(preparedStatement1);
+ when(session2.prepare(nullable(String.class))).thenReturn(preparedStatement2);
+
+ ResultSetFuture rsFuture = mock(ResultSetFuture.class);
+ ResultSet rs = mock(ResultSet.class);
+ Iterator it = mock(Iterator.class);
+ when(it.hasNext()).thenReturn(true);
+ when(it.next()).thenReturn(mock(Row.class));
+ when(rs.iterator()).thenReturn(it);
+ when(rsFuture.getUninterruptibly()).thenReturn(rs);
+ /* @formatter:off */
+ when(session1.executeAsync(any(Statement.class)))
+ .thenThrow(new InvalidQueryException("You may have used a PreparedStatement that was created with another Cluster instance"))
+ .thenThrow(new RuntimeException("this session should be refreshed / recreated"));
+ when(session2.executeAsync(boundStatement1))
+ .thenThrow(new InvalidQueryException("You may have used a PreparedStatement that was created with another Cluster instance"));
+ when(session2.executeAsync(boundStatement2)).thenReturn(rsFuture);
+ /* @formatter:on */
+
+ Cluster cluster = mock(Cluster.class);
+ when(cluster.connect()).thenReturn(session1).thenReturn(session2);
+ when(session1.getCluster()).thenReturn(cluster);
+ when(session2.getCluster()).thenReturn(cluster);
+
+ Cluster.Builder builder = mock(Cluster.Builder.class);
+ when(builder.build()).thenReturn(cluster);
+
+ CassandraSessionImpl cassandraSession = new CassandraSessionImpl(builder, null,
+ ConsistencyLevel.ONE, ConsistencyLevel.ONE, 0, mock(IgniteLogger.class));
+
+ BatchExecutionAssistant batchExecutionAssistant = new MyBatchExecutionAssistant();
+ ArrayList data = new ArrayList<>();
+ for (int i = 0; i < 10; i++) {
+ data.add(String.valueOf(i));
+ }
+ cassandraSession.execute(batchExecutionAssistant, data);
+
+ verify(cluster, times(2)).connect();
+ verify(session1, times(1)).prepare(nullable(String.class));
+ verify(session2, times(1)).prepare(nullable(String.class));
+ assertEquals(10, batchExecutionAssistant.processedCount());
+ }
+
+ /** */
+ private static PreparedStatement mockPreparedStatement() {
+ PreparedStatement ps = mock(PreparedStatement.class);
+ when(ps.getVariables()).thenReturn(mock(ColumnDefinitions.class));
+ when(ps.getPreparedId()).thenReturn(mock(PreparedId.class));
+ when(ps.getQueryString()).thenReturn("insert into xxx");
+ return ps;
+ }
+
+ /** */
+ private class MyBatchExecutionAssistant implements BatchExecutionAssistant {
+ /** */
+ private Set processed = new HashSet<>();
+
+ /** {@inheritDoc} */
+ @Override public void process(Row row, int seqNum) {
+ if (processed.contains(seqNum))
+ return;
+
+ processed.add(seqNum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean alreadyProcessed(int seqNum) {
+ return processed.contains(seqNum);
+ }
+
+ /** {@inheritDoc} */
+ @Override public int processedCount() {
+ return processed.size();
+ }
+
+ /** {@inheritDoc} */
+ @Override public boolean tableExistenceRequired() {
+ return false;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getTable() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String getStatement() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public BoundStatement bindStatement(PreparedStatement statement, Object obj) {
+ if (statement instanceof WrappedPreparedStatement)
+ statement = ((WrappedPreparedStatement)statement).getWrappedStatement();
+
+ if (statement == preparedStatement1) {
+ return boundStatement1;
+ }
+ else if (statement == preparedStatement2) {
+ return boundStatement2;
+ }
+
+ throw new RuntimeException("unexpected");
+ }
+
+ /** {@inheritDoc} */
+ @Override public KeyValuePersistenceSettings getPersistenceSettings() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public String operationName() {
+ return null;
+ }
+
+ /** {@inheritDoc} */
+ @Override public Object processedData() {
+ return null;
+ }
+
+ }
+
+ /** */
+ private static class MyBoundStatement1 extends BoundStatement {
+ /** */
+ MyBoundStatement1(PreparedStatement ps) {
+ super(ps);
+ }
+
+ }
+
+ /** */
+ private static class MyBoundStatement2 extends BoundStatement {
+ /** */
+ MyBoundStatement2(PreparedStatement ps) {
+ super(ps);
+ }
+ }
+
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java
new file mode 100644
index 000000000..60169e7c8
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DDLGeneratorTest.java
@@ -0,0 +1,60 @@
+/*
+ * 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.ignite.tests;
+
+import java.net.URL;
+import org.apache.ignite.cache.store.cassandra.utils.DDLGenerator;
+import org.junit.Test;
+
+/**
+ * DDLGenerator test.
+ */
+public class DDLGeneratorTest {
+ /** */
+ private static final String[] RESOURCES = new String[] {
+ "org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-4.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-5.xml",
+ "org/apache/ignite/tests/persistence/pojo/persistence-settings-6.xml",
+ "org/apache/ignite/tests/persistence/pojo/product.xml",
+ "org/apache/ignite/tests/persistence/pojo/order.xml"
+ };
+
+ /**
+ * Test DDL generator.
+ */
+ @Test
+ public void generatorTest() {
+ String[] files = new String[RESOURCES.length];
+
+ ClassLoader clsLdr = DDLGeneratorTest.class.getClassLoader();
+
+ for (int i = 0; i < RESOURCES.length; i++) {
+ URL url = clsLdr.getResource(RESOURCES[i]);
+ if (url == null)
+ throw new IllegalStateException("Failed to find resource: " + RESOURCES[i]);
+
+ files[i] = url.getFile();
+ }
+
+ DDLGenerator.main(files);
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DatasourceSerializationTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DatasourceSerializationTest.java
new file mode 100644
index 000000000..e981dea8c
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/DatasourceSerializationTest.java
@@ -0,0 +1,155 @@
+/*
+ * 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.ignite.tests;
+
+import java.io.Serializable;
+import java.lang.reflect.Field;
+import java.net.InetAddress;
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import com.datastax.driver.core.Cluster;
+import com.datastax.driver.core.ConsistencyLevel;
+import com.datastax.driver.core.Host;
+import com.datastax.driver.core.HostDistance;
+import com.datastax.driver.core.Statement;
+import com.datastax.driver.core.policies.LoadBalancingPolicy;
+import com.datastax.driver.core.policies.RoundRobinPolicy;
+import com.datastax.driver.core.policies.TokenAwarePolicy;
+import org.apache.ignite.cache.store.cassandra.datasource.Credentials;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.serializer.JavaSerializer;
+import org.apache.ignite.tests.utils.CassandraAdminCredentials;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test for datasource serialization.
+ */
+public class DatasourceSerializationTest {
+ /**
+ * Sample class for serialization test.
+ */
+ private static class MyLoadBalancingPolicy implements LoadBalancingPolicy, Serializable {
+ /** */
+ private transient LoadBalancingPolicy plc = new TokenAwarePolicy(new RoundRobinPolicy());
+
+ /** {@inheritDoc} */
+ @Override public void init(Cluster cluster, Collection hosts) {
+ plc.init(cluster, hosts);
+ }
+
+ /** {@inheritDoc} */
+ @Override public HostDistance distance(Host host) {
+ return plc.distance(host);
+ }
+
+ /** {@inheritDoc} */
+ @Override public Iterator newQueryPlan(String loggedKeyspace, Statement statement) {
+ return plc.newQueryPlan(loggedKeyspace, statement);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onAdd(Host host) {
+ plc.onAdd(host);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onUp(Host host) {
+ plc.onUp(host);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onDown(Host host) {
+ plc.onDown(host);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void onRemove(Host host) {
+ plc.onRemove(host);
+ }
+
+ /** {@inheritDoc} */
+ @Override public void close() {
+ plc.close();
+ }
+ }
+
+ /**
+ * Serialization test.
+ */
+ @Test
+ public void serializationTest() {
+ DataSource src = new DataSource();
+
+ Credentials cred = new CassandraAdminCredentials();
+ String[] points = new String[]{"127.0.0.1", "10.0.0.2", "10.0.0.3"};
+ LoadBalancingPolicy plc = new MyLoadBalancingPolicy();
+
+ src.setCredentials(cred);
+ src.setContactPoints(points);
+ src.setReadConsistency("ONE");
+ src.setWriteConsistency("QUORUM");
+ src.setLoadBalancingPolicy(plc);
+
+ JavaSerializer serializer = new JavaSerializer();
+
+ ByteBuffer buff = serializer.serialize(src);
+ DataSource _src = (DataSource)serializer.deserialize(buff);
+
+ Credentials _cred = (Credentials)getFieldValue(_src, "creds");
+ List _points = (List)getFieldValue(_src, "contactPoints");
+ ConsistencyLevel _readCons = (ConsistencyLevel)getFieldValue(_src, "readConsistency");
+ ConsistencyLevel _writeCons = (ConsistencyLevel)getFieldValue(_src, "writeConsistency");
+ LoadBalancingPolicy _plc = (LoadBalancingPolicy)getFieldValue(_src, "loadBalancingPlc");
+
+ assertTrue("Incorrectly serialized/deserialized credentials for Cassandra DataSource",
+ cred.getPassword().equals(_cred.getPassword()) && cred.getUser().equals(_cred.getUser()));
+
+ assertTrue("Incorrectly serialized/deserialized contact points for Cassandra DataSource",
+ "/127.0.0.1".equals(_points.get(0).toString()) &&
+ "/10.0.0.2".equals(_points.get(1).toString()) &&
+ "/10.0.0.3".equals(_points.get(2).toString()));
+
+ assertTrue("Incorrectly serialized/deserialized consistency levels for Cassandra DataSource",
+ ConsistencyLevel.ONE == _readCons && ConsistencyLevel.QUORUM == _writeCons);
+
+ assertTrue("Incorrectly serialized/deserialized load balancing policy for Cassandra DataSource",
+ _plc instanceof MyLoadBalancingPolicy);
+ }
+
+ /**
+ * @param obj Object.
+ * @param field Field name.
+ * @return Field value.
+ */
+ private Object getFieldValue(Object obj, String field) {
+ try {
+ Field f = obj.getClass().getDeclaredField(field);
+
+ f.setAccessible(true);
+
+ return f.get(obj);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Failed to get field '" + field + "' value", e);
+ }
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java
new file mode 100644
index 000000000..9a759bc3c
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.ignite.tests;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.tests.load.LoadTestDriver;
+import org.apache.ignite.tests.load.ignite.BulkReadWorker;
+import org.apache.ignite.tests.load.ignite.BulkWriteWorker;
+import org.apache.ignite.tests.load.ignite.ReadWorker;
+import org.apache.ignite.tests.load.ignite.WriteWorker;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+/**
+ * Load tests for Ignite caches which utilizing {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}
+ * to store cache data into Cassandra tables
+ */
+public class IgnitePersistentStoreLoadTest extends LoadTestDriver {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger("IgniteLoadTests");
+
+ /**
+ * test starter.
+ *
+ * @param args Test arguments.
+ */
+ public static void main(String[] args) {
+ try {
+ LOGGER.info("Ignite load tests execution started");
+
+ LoadTestDriver driver = new IgnitePersistentStoreLoadTest();
+
+ /**
+ * Load test scripts could be executed from several machines. Current implementation can correctly,
+ * handle situation when Cassandra keyspace/table was dropped - for example by the same load test
+ * started a bit later on another machine. Moreover there is a warm up period for each load test.
+ * Thus all the delays related to keyspaces/tables recreation actions will not affect performance metrics,
+ * but it will be produced lots of "trash" output in the logs (related to correct handling of such
+ * exceptional situation and keyspace/table recreation).
+ *
+ * Thus dropping test keyspaces makes sense only for Unit tests, but not for Load tests.
+ **/
+
+ //CassandraHelper.dropTestKeyspaces();
+
+ driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME);
+
+ driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME);
+
+ driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME);
+
+ driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME);
+
+ /**
+ * Load test script executed on one machine could complete earlier that the same load test executed from
+ * another machine. Current implementation can correctly handle situation when Cassandra keyspace/table
+ * was dropped (simply recreate it). But dropping keyspace/table during load tests execution and subsequent
+ * recreation of such objects can have SIGNIFICANT EFFECT on final performance metrics.
+ *
+ * Thus dropping test keyspaces at the end of the tests makes sense only for Unit tests,
+ * but not for Load tests.
+ */
+
+ //CassandraHelper.dropTestKeyspaces();
+
+ LOGGER.info("Ignite load tests execution completed");
+ }
+ catch (Throwable e) {
+ LOGGER.error("Ignite load tests execution failed", e);
+ throw new RuntimeException("Ignite load tests execution failed", e);
+ }
+ finally {
+ CassandraHelper.releaseCassandraResources();
+ }
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Logger logger() {
+ return LOGGER;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected Object setup(String logName) {
+ return Ignition.start(TestsHelper.getLoadTestsIgniteConfig());
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void tearDown(Object obj) {
+ Ignite ignite = (Ignite)obj;
+
+ if (ignite != null)
+ ignite.close();
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStorePrimitiveTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStorePrimitiveTest.java
new file mode 100644
index 000000000..5b2799a2b
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStorePrimitiveTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.ignite.tests;
+
+import java.io.IOException;
+import java.net.URL;
+import com.datastax.driver.core.policies.RoundRobinPolicy;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.tests.utils.CassandraAdminCredentials;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Unit test for Ignite caches which utilizing {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}
+ * to store primitive type cache data into Cassandra table.
+ */
+public class IgnitePersistentStorePrimitiveTest {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger(IgnitePersistentStorePrimitiveTest.class.getName());
+
+ /** */
+ @BeforeClass
+ public static void setUpClass() {
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.startEmbeddedCassandra(LOGGER);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+ }
+ }
+
+ LOGGER.info("Testing admin connection to Cassandra");
+ CassandraHelper.testAdminConnection();
+
+ LOGGER.info("Testing regular connection to Cassandra");
+ CassandraHelper.testRegularConnection();
+
+ LOGGER.info("Dropping all artifacts from previous tests execution session");
+ CassandraHelper.dropTestKeyspaces();
+
+ LOGGER.info("Start tests execution");
+ }
+
+ /** */
+ @AfterClass
+ public static void tearDownClass() {
+ try {
+ CassandraHelper.dropTestKeyspaces();
+ }
+ finally {
+ CassandraHelper.releaseCassandraResources();
+
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.stopEmbeddedCassandra();
+ }
+ catch (Throwable e) {
+ LOGGER.error("Failed to stop embedded Cassandra instance", e);
+ }
+ }
+ }
+ }
+
+ /** */
+ @Test
+ public void test() throws IOException {
+ IgniteConfiguration config = igniteConfig();
+
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start(config)) {
+ IgniteCache cache = ignite.getOrCreateCache("cache1");
+ cache.put(12L, 12L);
+ }
+
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start(config)) {
+ IgniteCache cache = ignite.getOrCreateCache("cache1");
+
+ assertEquals(12L, (long)cache.get(12L));
+
+ cache.remove(12L);
+ }
+ }
+
+ /** */
+ private IgniteConfiguration igniteConfig() throws IOException {
+ URL url = getClass().getClassLoader().getResource("org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml");
+ String persistence = U.readFileToString(url.getFile(), "UTF-8");
+ KeyValuePersistenceSettings persistenceSettings = new KeyValuePersistenceSettings(persistence);
+
+ DataSource dataSource = new DataSource();
+ dataSource.setContactPoints(CassandraHelper.getContactPointsArray());
+ dataSource.setCredentials(new CassandraAdminCredentials());
+ dataSource.setLoadBalancingPolicy(new RoundRobinPolicy());
+
+ CassandraCacheStoreFactory storeFactory = new CassandraCacheStoreFactory<>();
+ storeFactory.setDataSource(dataSource);
+ storeFactory.setPersistenceSettings(persistenceSettings);
+
+ CacheConfiguration cacheConfiguration = new CacheConfiguration<>();
+ cacheConfiguration.setName("cache1");
+ cacheConfiguration.setReadThrough(true);
+ cacheConfiguration.setWriteThrough(true);
+ cacheConfiguration.setCacheStoreFactory(storeFactory);
+
+ IgniteConfiguration config = new IgniteConfiguration();
+ config.setCacheConfiguration(cacheConfiguration);
+
+ return config;
+ }
+}
diff --git a/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
new file mode 100644
index 000000000..1aeade57a
--- /dev/null
+++ b/modules/cassandra-ext/store/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java
@@ -0,0 +1,920 @@
+/*
+ * 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.ignite.tests;
+
+import java.io.IOException;
+import java.net.URL;
+import java.time.Instant;
+import java.util.Collection;
+import java.util.Map;
+import com.datastax.driver.core.SimpleStatement;
+import com.datastax.driver.core.policies.RoundRobinPolicy;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteTransactions;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CachePeekMode;
+import org.apache.ignite.cache.store.CacheStore;
+import org.apache.ignite.cache.store.cassandra.CassandraCacheStoreFactory;
+import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
+import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.tests.pojos.Person;
+import org.apache.ignite.tests.pojos.PersonId;
+import org.apache.ignite.tests.pojos.Product;
+import org.apache.ignite.tests.pojos.ProductOrder;
+import org.apache.ignite.tests.pojos.SimplePerson;
+import org.apache.ignite.tests.pojos.SimplePersonId;
+import org.apache.ignite.tests.utils.CacheStoreHelper;
+import org.apache.ignite.tests.utils.CassandraAdminCredentials;
+import org.apache.ignite.tests.utils.CassandraHelper;
+import org.apache.ignite.tests.utils.TestsHelper;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.springframework.core.io.ClassPathResource;
+
+/**
+ * Unit tests for Ignite caches which utilizing {@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore}
+ * to store cache data into Cassandra tables
+ */
+public class IgnitePersistentStoreTest {
+ /** */
+ private static final Logger LOGGER = LogManager.getLogger(IgnitePersistentStoreTest.class.getName());
+
+ /** */
+ @BeforeClass
+ public static void setUpClass() {
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.startEmbeddedCassandra(LOGGER);
+ }
+ catch (Throwable e) {
+ throw new RuntimeException("Failed to start embedded Cassandra instance", e);
+ }
+ }
+
+ LOGGER.info("Testing admin connection to Cassandra");
+ CassandraHelper.testAdminConnection();
+
+ LOGGER.info("Testing regular connection to Cassandra");
+ CassandraHelper.testRegularConnection();
+
+ LOGGER.info("Dropping all artifacts from previous tests execution session");
+ CassandraHelper.dropTestKeyspaces();
+
+ LOGGER.info("Start tests execution");
+ }
+
+ /** */
+ @AfterClass
+ public static void tearDownClass() {
+ try {
+ CassandraHelper.dropTestKeyspaces();
+ }
+ finally {
+ CassandraHelper.releaseCassandraResources();
+
+ if (CassandraHelper.useEmbeddedCassandra()) {
+ try {
+ CassandraHelper.stopEmbeddedCassandra();
+ }
+ catch (Throwable e) {
+ LOGGER.error("Failed to stop embedded Cassandra instance", e);
+ }
+ }
+ }
+ }
+
+ /** */
+ @Test
+ public void primitiveStrategyTest() {
+ Ignition.stopAll(true);
+
+ Map longMap = TestsHelper.generateLongsMap();
+ Map strMap = TestsHelper.generateStringsMap();
+
+ LOGGER.info("Running PRIMITIVE strategy write tests");
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/primitive/ignite-config.xml")) {
+ IgniteCache longCache = ignite.getOrCreateCache(new CacheConfiguration("cache1"));
+ IgniteCache strCache = ignite.getOrCreateCache(new CacheConfiguration("cache2"));
+
+ LOGGER.info("Running single operation write tests");
+ longCache.put(1L, 1L);
+ strCache.put("1", "1");
+ LOGGER.info("Single operation write tests passed");
+
+ LOGGER.info("Running bulk operation write tests");
+ longCache.putAll(longMap);
+ strCache.putAll(strMap);
+ LOGGER.info("Bulk operation write tests passed");
+ }
+
+ LOGGER.info("PRIMITIVE strategy write tests passed");
+
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/primitive/ignite-config.xml")) {
+ LOGGER.info("Running PRIMITIVE strategy read tests");
+
+ IgniteCache longCache = ignite.getOrCreateCache(new CacheConfiguration("cache1"));
+ IgniteCache strCache = ignite.getOrCreateCache(new CacheConfiguration("cache2"));
+
+ LOGGER.info("Running single operation read tests");
+
+ Long longVal = longCache.get(1L);
+ if (!longVal.equals(longMap.get(1L)))
+ throw new RuntimeException("Long value was incorrectly deserialized from Cassandra");
+
+ String strVal = strCache.get("1");
+ if (!strVal.equals(strMap.get("1")))
+ throw new RuntimeException("String value was incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Single operation read tests passed");
+
+ LOGGER.info("Running bulk operation read tests");
+
+ Map longMap1 = longCache.getAll(longMap.keySet());
+ if (!TestsHelper.checkMapsEqual(longMap, longMap1))
+ throw new RuntimeException("Long values batch was incorrectly deserialized from Cassandra");
+
+ Map strMap1 = strCache.getAll(strMap.keySet());
+ if (!TestsHelper.checkMapsEqual(strMap, strMap1))
+ throw new RuntimeException("String values batch was incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk operation read tests passed");
+
+ LOGGER.info("PRIMITIVE strategy read tests passed");
+
+ LOGGER.info("Running PRIMITIVE strategy delete tests");
+
+ longCache.remove(1L);
+ longCache.removeAll(longMap.keySet());
+
+ strCache.remove("1");
+ strCache.removeAll(strMap.keySet());
+
+ LOGGER.info("PRIMITIVE strategy delete tests passed");
+ }
+ }
+
+ /** */
+ @Test
+ public void blobStrategyTest() {
+ Ignition.stopAll(true);
+
+ Map longMap = TestsHelper.generateLongsMap();
+ Map personMap = TestsHelper.generateLongsPersonsMap();
+
+ LOGGER.info("Running BLOB strategy write tests");
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/blob/ignite-config.xml")) {
+ IgniteCache longCache = ignite.getOrCreateCache(new CacheConfiguration("cache1"));
+ IgniteCache personCache = ignite.getOrCreateCache(new CacheConfiguration("cache2"));
+
+ LOGGER.info("Running single operation write tests");
+ longCache.put(1L, 1L);
+ personCache.put(1L, TestsHelper.generateRandomPerson(1L));
+ LOGGER.info("Single operation write tests passed");
+
+ LOGGER.info("Running bulk operation write tests");
+ longCache.putAll(longMap);
+ personCache.putAll(personMap);
+ LOGGER.info("Bulk operation write tests passed");
+ }
+
+ LOGGER.info("BLOB strategy write tests passed");
+
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/blob/ignite-config.xml")) {
+ LOGGER.info("Running BLOB strategy read tests");
+
+ IgniteCache longCache = ignite.getOrCreateCache(new CacheConfiguration("cache1"));
+ IgniteCache personCache = ignite.getOrCreateCache(new CacheConfiguration("cache2"));
+
+ LOGGER.info("Running single operation read tests");
+
+ Long longVal = longCache.get(1L);
+ if (!longVal.equals(longMap.get(1L)))
+ throw new RuntimeException("Long value was incorrectly deserialized from Cassandra");
+
+ Person person = personCache.get(1L);
+ if (!person.equals(personMap.get(1L)))
+ throw new RuntimeException("Person value was incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Single operation read tests passed");
+
+ LOGGER.info("Running bulk operation read tests");
+
+ Map longMap1 = longCache.getAll(longMap.keySet());
+ if (!TestsHelper.checkMapsEqual(longMap, longMap1))
+ throw new RuntimeException("Long values batch was incorrectly deserialized from Cassandra");
+
+ Map personMap1 = personCache.getAll(personMap.keySet());
+ if (!TestsHelper.checkPersonMapsEqual(personMap, personMap1, false))
+ throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra");
+
+ LOGGER.info("Bulk operation read tests passed");
+
+ LOGGER.info("BLOB strategy read tests passed");
+
+ LOGGER.info("Running BLOB strategy delete tests");
+
+ longCache.remove(1L);
+ longCache.removeAll(longMap.keySet());
+
+ personCache.remove(1L);
+ personCache.removeAll(personMap.keySet());
+
+ LOGGER.info("BLOB strategy delete tests passed");
+ }
+ }
+
+ /** */
+ @Test
+ public void blobBinaryLoadCacheTest() {
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) {
+ IgniteCache personCache = ignite.getOrCreateCache("cache2");
+
+ assert ignite.configuration().getMarshaller() instanceof BinaryMarshaller;
+
+ personCache.put(1L, new PojoPerson(1, "name"));
+
+ assert personCache.withKeepBinary().get(1L) instanceof BinaryObject;
+ }
+
+ Ignition.stopAll(true);
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/loadall_blob/ignite-config.xml")) {
+ IgniteCache personCache = ignite.getOrCreateCache("cache2");
+
+ personCache.loadCache(null, null);
+
+ PojoPerson person = personCache.get(1L);
+
+ LOGGER.info("loadCache tests passed");
+ }
+ }
+
+ /** */
+ @Test
+ public void pojoStrategyTest() {
+ Ignition.stopAll(true);
+
+ LOGGER.info("Running POJO strategy write tests");
+
+ Map personMap1 = TestsHelper.generateLongsPersonsMap();
+ Map personMap2 = TestsHelper.generatePersonIdsPersonsMap();
+ Map productsMap = TestsHelper.generateProductsMap();
+ Map ordersMap = TestsHelper.generateOrdersMap();
+
+ Product product = TestsHelper.generateRandomProduct(-1L);
+ ProductOrder order = TestsHelper.generateRandomOrder(-1L);
+
+ try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) {
+ IgniteCache