From bc79884289a567d99be3139702e0c6a7903f1ff2 Mon Sep 17 00:00:00 2001 From: root Date: Fri, 16 Oct 2015 07:46:03 +0200 Subject: [PATCH] # ignite-1371 --- modules/cassandra/README.txt | 32 ++ modules/cassandra/licenses/apache-2.0.txt | 202 ++++++++ modules/cassandra/pom.xml | 211 +++++++++ .../cache/store/cassandra/CassandraCacheStore.java | 329 +++++++++++++ .../cassandra/CassandraCacheStoreFactory.java | 123 +++++ .../cassandra/utils/common/CassandraHelper.java | 111 +++++ .../utils/common/PropertyMappingHelper.java | 158 +++++++ .../store/cassandra/utils/common/SystemHelper.java | 48 ++ .../cassandra/utils/datasource/Credentials.java | 27 ++ .../cassandra/utils/datasource/DataSource.java | 348 ++++++++++++++ .../utils/datasource/PlainCredentials.java | 40 ++ .../utils/persistence/KeyPersistenceSettings.java | 223 +++++++++ .../persistence/KeyValuePersistenceSettings.java | 356 ++++++++++++++ .../utils/persistence/PersistenceController.java | 303 ++++++++++++ .../utils/persistence/PersistenceSettings.java | 252 ++++++++++ .../utils/persistence/PersistenceStrategy.java | 61 +++ .../cassandra/utils/persistence/PojoField.java | 151 ++++++ .../cassandra/utils/persistence/PojoKeyField.java | 65 +++ .../utils/persistence/PojoValueField.java | 104 +++++ .../persistence/ValuePersistenceSettings.java | 91 ++++ .../cassandra/utils/serializer/JavaSerializer.java | 99 ++++ .../cassandra/utils/serializer/KryoSerializer.java | 105 +++++ .../cassandra/utils/serializer/Serializer.java | 31 ++ .../utils/session/BatchExecutionAssistant.java | 49 ++ .../cassandra/utils/session/CassandraSession.java | 32 ++ .../utils/session/CassandraSessionImpl.java | 516 +++++++++++++++++++++ .../utils/session/ExecutionAssistant.java | 42 ++ .../session/GenericBatchExecutionAssistant.java | 59 +++ .../cassandra/utils/session/pool/SessionPool.java | 151 ++++++ .../utils/session/pool/SessionWrapper.java | 49 ++ .../tests/CassandraDirectPersistenceLoadTest.java | 76 +++ .../tests/CassandraDirectPersistenceTest.java | 321 +++++++++++++ .../tests/IgnitePersistentStoreLoadTest.java | 80 ++++ .../ignite/tests/IgnitePersistentStoreTest.java | 317 +++++++++++++ .../org/apache/ignite/tests/load/Generator.java | 26 ++ .../org/apache/ignite/tests/load/IntGenerator.java | 28 ++ .../apache/ignite/tests/load/LoadTestDriver.java | 146 ++++++ .../apache/ignite/tests/load/PersonGenerator.java | 41 ++ .../ignite/tests/load/PersonIdGenerator.java | 30 ++ .../apache/ignite/tests/load/StringGenerator.java | 27 ++ .../java/org/apache/ignite/tests/load/Worker.java | 307 ++++++++++++ .../tests/load/cassandra/BulkReadWorker.java | 57 +++ .../tests/load/cassandra/BulkWriteWorker.java | 47 ++ .../ignite/tests/load/cassandra/ReadWorker.java | 46 ++ .../ignite/tests/load/cassandra/WriteWorker.java | 46 ++ .../ignite/tests/load/ignite/BulkReadWorker.java | 47 ++ .../ignite/tests/load/ignite/BulkWriteWorker.java | 47 ++ .../ignite/tests/load/ignite/ReadWorker.java | 46 ++ .../ignite/tests/load/ignite/WriteWorker.java | 46 ++ .../java/org/apache/ignite/tests/pojos/Person.java | 215 +++++++++ .../org/apache/ignite/tests/pojos/PersonId.java | 97 ++++ .../ignite/tests/utils/CacheStoreHelper.java | 60 +++ .../tests/utils/CassandraAdminCredentials.java | 34 ++ .../apache/ignite/tests/utils/CassandraHelper.java | 281 +++++++++++ .../tests/utils/CassandraRegularCredentials.java | 34 ++ .../ignite/tests/utils/TestCacheSession.java | 77 +++ .../org/apache/ignite/tests/utils/TestsHelper.java | 312 +++++++++++++ .../cassandra/src/test/resources/log4j.properties | 104 +++++ .../ignite/tests/cassandra/connection-settings.xml | 30 ++ .../ignite/tests/cassandra/connection.properties | 2 + .../ignite/tests/cassandra/credentials.properties | 7 + .../ignite/tests/cassandra/keyspaces.properties | 2 + .../tests/persistence/blob/ignite-config.xml | 82 ++++ .../persistence/blob/persistence-settings-1.xml | 4 + .../persistence/blob/persistence-settings-2.xml | 4 + .../persistence/blob/persistence-settings-3.xml | 12 + .../tests/persistence/pojo/ignite-config.xml | 101 ++++ .../persistence/pojo/persistence-settings-1.xml | 4 + .../persistence/pojo/persistence-settings-2.xml | 4 + .../persistence/pojo/persistence-settings-3.xml | 31 ++ .../tests/persistence/primitive/ignite-config.xml | 82 ++++ .../primitive/ignite-remote-client-config.xml | 75 +++ .../primitive/ignite-remote-server-config.xml | 88 ++++ .../primitive/persistence-settings-1.xml | 4 + .../primitive/persistence-settings-2.xml | 4 + .../cassandra/src/test/resources/tests.properties | 31 ++ .../src/test/scripts/cassandra-load-tests.bat | 37 ++ .../src/test/scripts/cassandra-load-tests.sh | 37 ++ .../src/test/scripts/ignite-load-tests.bat | 37 ++ .../src/test/scripts/ignite-load-tests.sh | 37 ++ pom.xml | 1 + 81 files changed, 8079 insertions(+) create mode 100644 modules/cassandra/README.txt create mode 100644 modules/cassandra/licenses/apache-2.0.txt create mode 100644 modules/cassandra/pom.xml create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/CassandraHelper.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/PropertyMappingHelper.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/SystemHelper.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/Credentials.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/DataSource.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/PlainCredentials.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyPersistenceSettings.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyValuePersistenceSettings.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceController.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceSettings.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceStrategy.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoField.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoKeyField.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoValueField.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/ValuePersistenceSettings.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/JavaSerializer.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/KryoSerializer.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/Serializer.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/BatchExecutionAssistant.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSession.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSessionImpl.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/ExecutionAssistant.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/GenericBatchExecutionAssistant.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionPool.java create mode 100644 modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionWrapper.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java create mode 100644 modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java create mode 100644 modules/cassandra/src/test/resources/log4j.properties create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml create mode 100644 modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml create mode 100644 modules/cassandra/src/test/resources/tests.properties create mode 100644 modules/cassandra/src/test/scripts/cassandra-load-tests.bat create mode 100644 modules/cassandra/src/test/scripts/cassandra-load-tests.sh create mode 100644 modules/cassandra/src/test/scripts/ignite-load-tests.bat create mode 100644 modules/cassandra/src/test/scripts/ignite-load-tests.sh diff --git a/modules/cassandra/README.txt b/modules/cassandra/README.txt new file mode 100644 index 0000000..dde0a2f --- /dev/null +++ b/modules/cassandra/README.txt @@ -0,0 +1,32 @@ +Apache Ignite Cassandra Module +------------------------ + +Apache Ignite Cassandra module provides CacheStore implementation backed by Cassandra database. + +To enable Cassandra module when starting a standalone node, move 'optional/ignite-cloud' folder to +'libs' folder before running 'ignite.{sh|bat}' script. The content of the module folder will +be added to classpath in this case. + +Importing Cassandra Module In Maven Project +------------------------------------- + +If you are using Maven to manage dependencies of your project, you can add Cassandra module +dependency like this (replace '${ignite.version}' with actual Ignite version you are +interested in): + + + ... + + ... + + org.apache.ignite + ignite-cassandra + ${ignite.version} + + ... + + ... + diff --git a/modules/cassandra/licenses/apache-2.0.txt b/modules/cassandra/licenses/apache-2.0.txt new file mode 100644 index 0000000..d645695 --- /dev/null +++ b/modules/cassandra/licenses/apache-2.0.txt @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 + http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, to cause the + direction or management of such entity, whether by contract or + otherwise, or (ii) ownership of fifty percent (50%) or more of the + outstanding shares, or (iii) beneficial ownership of such entity. + + "You" (or "Your") shall mean an individual or Legal Entity + exercising permissions granted by this License. + + "Source" form shall mean the preferred form for making modifications, + including but not limited to software source code, documentation + source, and configuration files. + + "Object" form shall mean any form resulting from mechanical + transformation or translation of a Source form, including but + not limited to compiled object code, generated documentation, + and conversions to other media types. + + "Work" shall mean the work of authorship, whether in Source or + Object form, made available under the License, as indicated by a + copyright notice that is included in or attached to the work + (an example is provided in the Appendix below). + + "Derivative Works" shall mean any work, whether in Source or Object + form, that is based on (or derived from) the Work and for which the + editorial revisions, annotations, elaborations, or other modifications + represent, as a whole, an original work of authorship. For the purposes + of this License, Derivative Works shall not include works that remain + separable from, or merely link (or bind by name) to the interfaces of, + the Work and Derivative Works thereof. + + "Contribution" shall mean any work of authorship, including + the original version of the Work and any modifications or additions + to that Work or Derivative Works thereof, that is intentionally + submitted to Licensor for inclusion in the Work by the copyright owner + or by an individual or Legal Entity authorized to submit on behalf of + the copyright owner. For the purposes of this definition, "submitted" + means any form of electronic, verbal, or written communication sent + to the Licensor or its representatives, including but not limited to + communication on electronic mailing lists, source code control systems, + and issue tracking systems that are managed by, or on behalf of, the + Licensor for the purpose of discussing and improving the Work, but + excluding communication that is conspicuously marked or otherwise + designated in writing by the copyright owner as "Not a Contribution." + + "Contributor" shall mean Licensor and any individual or Legal Entity + on behalf of whom a Contribution has been received by Licensor and + subsequently incorporated within the Work. + + 2. Grant of Copyright License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + copyright license to reproduce, prepare Derivative Works of, + publicly display, publicly perform, sublicense, and distribute the + Work and such Derivative Works in Source or Object form. + + 3. Grant of Patent License. Subject to the terms and conditions of + this License, each Contributor hereby grants to You a perpetual, + worldwide, non-exclusive, no-charge, royalty-free, irrevocable + (except as stated in this section) patent license to make, have made, + use, offer to sell, sell, import, and otherwise transfer the Work, + where such license applies only to those patent claims licensable + by such Contributor that are necessarily infringed by their + Contribution(s) alone or by combination of their Contribution(s) + with the Work to which such Contribution(s) was submitted. If You + institute patent litigation against any entity (including a + cross-claim or counterclaim in a lawsuit) alleging that the Work + or a Contribution incorporated within the Work constitutes direct + or contributory patent infringement, then any patent licenses + granted to You under this License for that Work shall terminate + as of the date such litigation is filed. + + 4. Redistribution. You may reproduce and distribute copies of the + Work or Derivative Works thereof in any medium, with or without + modifications, and in Source or Object form, provided that You + meet the following conditions: + + (a) You must give any other recipients of the Work or + Derivative Works a copy of this License; and + + (b) You must cause any modified files to carry prominent notices + stating that You changed the files; and + + (c) You must retain, in the Source form of any Derivative Works + that You distribute, all copyright, patent, trademark, and + attribution notices from the Source form of the Work, + excluding those notices that do not pertain to any part of + the Derivative Works; and + + (d) If the Work includes a "NOTICE" text file as part of its + distribution, then any Derivative Works that You distribute must + include a readable copy of the attribution notices contained + within such NOTICE file, excluding those notices that do not + pertain to any part of the Derivative Works, in at least one + of the following places: within a NOTICE text file distributed + as part of the Derivative Works; within the Source form or + documentation, if provided along with the Derivative Works; or, + within a display generated by the Derivative Works, if and + wherever such third-party notices normally appear. The contents + of the NOTICE file are for informational purposes only and + do not modify the License. You may add Your own attribution + notices within Derivative Works that You distribute, alongside + or as an addendum to the NOTICE text from the Work, provided + that such additional attribution notices cannot be construed + as modifying the License. + + You may add Your own copyright statement to Your modifications and + may provide additional or different license terms and conditions + for use, reproduction, or distribution of Your modifications, or + for any such Derivative Works as a whole, provided Your use, + reproduction, and distribution of the Work otherwise complies with + the conditions stated in this License. + + 5. Submission of Contributions. Unless You explicitly state otherwise, + any Contribution intentionally submitted for inclusion in the Work + by You to the Licensor shall be under the terms and conditions of + this License, without any additional terms or conditions. + Notwithstanding the above, nothing herein shall supersede or modify + the terms of any separate license agreement you may have executed + with Licensor regarding such Contributions. + + 6. Trademarks. This License does not grant permission to use the trade + names, trademarks, service marks, or product names of the Licensor, + except as required for reasonable and customary use in describing the + origin of the Work and reproducing the content of the NOTICE file. + + 7. Disclaimer of Warranty. Unless required by applicable law or + agreed to in writing, Licensor provides the Work (and each + Contributor provides its Contributions) on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or + implied, including, without limitation, any warranties or conditions + of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A + PARTICULAR PURPOSE. You are solely responsible for determining the + appropriateness of using or redistributing the Work and assume any + risks associated with Your exercise of permissions under this License. + + 8. Limitation of Liability. In no event and under no legal theory, + whether in tort (including negligence), contract, or otherwise, + unless required by applicable law (such as deliberate and grossly + negligent acts) or agreed to in writing, shall any Contributor be + liable to You for damages, including any direct, indirect, special, + incidental, or consequential damages of any character arising as a + result of this License or out of the use or inability to use the + Work (including but not limited to damages for loss of goodwill, + work stoppage, computer failure or malfunction, or any and all + other commercial damages or losses), even if such Contributor + has been advised of the possibility of such damages. + + 9. Accepting Warranty or Additional Liability. While redistributing + the Work or Derivative Works thereof, You may choose to offer, + and charge a fee for, acceptance of support, warranty, indemnity, + or other liability obligations and/or rights consistent with this + License. However, in accepting such obligations, You may act only + on Your own behalf and on Your sole responsibility, not on behalf + of any other Contributor, and only if You agree to indemnify, + defend, and hold each Contributor harmless for any liability + incurred by, or claims asserted against, such Contributor by reason + of your accepting any such warranty or additional liability. + + END OF TERMS AND CONDITIONS + + APPENDIX: How to apply the Apache License to your work. + + To apply the Apache License to your work, attach the following + boilerplate notice, with the fields enclosed by brackets "[]" + replaced with your own identifying information. (Don't include + the brackets!) The text should be enclosed in the appropriate + comment syntax for the file format. We also recommend that a + file or class name and description of purpose be included on the + same "printed page" as the copyright notice for easier + identification within third-party archives. + + Copyright [yyyy] [name of copyright owner] + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. diff --git a/modules/cassandra/pom.xml b/modules/cassandra/pom.xml new file mode 100644 index 0000000..f7556e8 --- /dev/null +++ b/modules/cassandra/pom.xml @@ -0,0 +1,211 @@ + + + + + + + 4.0.0 + + + org.apache.ignite + ignite-parent + 1 + ../../parent + + + ignite-cassandra + 1.4.0-SNAPSHOT + http://ignite.apache.org + + + 1.8.3 + 4.12 + 1.2.17 + 2.1.7.1 + 3.0.3 + + + + + + commons-beanutils + commons-beanutils + ${commons-beanutils.version} + + + + + com.esotericsoftware + kryo + ${kryo.version} + + + + + org.apache.ignite + ignite-core + ${project.version} + + + + org.apache.ignite + ignite-spring + ${project.version} + + + + org.apache.ignite + ignite-log4j + ${project.version} + test + + + + + com.datastax.cassandra + cassandra-driver-core + ${cassandra.version} + + + + + junit + junit + ${junit.version} + test + + + + + log4j + log4j + ${log4j.version} + test + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.2 + + 1.7 + 1.7 + 1.7 + UTF-8 + true + false + lines,vars,source + 256 + 512 + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.10 + + + copy-all-dependencies + package + + copy-dependencies + + + ${project.build.directory}/tests-package/lib + false + false + true + + + + copy-main-dependencies + package + + copy-dependencies + + + ${project.build.directory}/libs + false + false + true + + + org.apache.ignite,org.springframework,org.gridgain + + + commons-logging,slf4j-api,cache-api,slf4j-api,aopalliance + + runtime + + + + + + + org.apache.maven.plugins + maven-antrun-plugin + 1.8 + + + package-tests + package + + run + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java new file mode 100644 index 0000000..0c057f1 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java @@ -0,0 +1,329 @@ +/* + * 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; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Row; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import javax.cache.Cache; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreSession; +import org.apache.ignite.cache.store.cassandra.utils.datasource.DataSource; +import org.apache.ignite.cache.store.cassandra.utils.persistence.KeyValuePersistenceSettings; +import org.apache.ignite.cache.store.cassandra.utils.persistence.PersistenceController; +import org.apache.ignite.cache.store.cassandra.utils.session.CassandraSession; +import org.apache.ignite.cache.store.cassandra.utils.session.GenericBatchExecutionAssistant; +import org.apache.ignite.cache.store.cassandra.utils.session.ExecutionAssistant; +import org.apache.ignite.lang.IgniteBiInClosure; +import org.apache.ignite.logger.NullLogger; +import org.apache.ignite.resources.CacheStoreSessionResource; +import org.apache.ignite.resources.LoggerResource; + +/** + * ${@link org.apache.ignite.cache.store.CacheStore} implementation to store Ignite cache key/value into Cassandra database + * @param Ignite cache key type + * @param Ignite cache value type + */ +public class CassandraCacheStore implements CacheStore { + private static final String ATTR_CONN_PROP = "CASSANDRA_STORE_CONNECTION"; + + /** Auto-injected store session. */ + @CacheStoreSessionResource + private CacheStoreSession storeSession; + + /** Auto-injected logger instance. */ + @LoggerResource + private IgniteLogger logger; + + private DataSource dataSource; + private PersistenceController controller; + + public CassandraCacheStore(DataSource dataSource, KeyValuePersistenceSettings settings) { + this.dataSource = dataSource; + this.controller = new PersistenceController(settings); + } + + @Override public void loadCache(IgniteBiInClosure closure, Object... args) throws CacheLoaderException { + } + + @Override public void sessionEnd(boolean commit) throws CacheWriterException { + if (storeSession == null || storeSession.transaction() == null) + return; + + CassandraSession cassandraSession = (CassandraSession)storeSession.properties().remove(ATTR_CONN_PROP); + if (cassandraSession != null) { + try { + cassandraSession.close(); + } + catch (Throwable ignored) { + } + } + } + + @SuppressWarnings({"unchecked"}) + @Override public V load(final K key) throws CacheLoaderException { + if (key == null) + return null; + + CassandraSession ses = getCassandraSession(); + + try { + return ses.execute(new ExecutionAssistant() { + @Override public boolean tableExistenceRequired() { + return false; + } + + @Override public String getStatement() { + return controller.getLoadStatement(false); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement) { + return controller.bindKey(statement, key); + } + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "READ"; + } + + @Override public V process(Row row) { + return row == null ? null : (V)controller.buildValueObject(row); + } + }); + } + finally { + closeCassandraSession(ses); + } + } + + @SuppressWarnings("unchecked") + @Override public Map loadAll(Iterable keys) throws CacheLoaderException { + if (keys == null || !keys.iterator().hasNext()) + return new HashMap<>(); + + CassandraSession ses = getCassandraSession(); + + try { + return ses.execute(new GenericBatchExecutionAssistant, K>() { + private Map data = new HashMap<>(); + + @Override public String getStatement() { + return controller.getLoadStatement(true); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement, K key) { + return controller.bindKey(statement, key); + } + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "BULK_READ"; + } + + @Override public Map processedData() { + return data; + } + + @Override protected void process(Row row) { + data.put((K)controller.buildKeyObject(row), (V)controller.buildValueObject(row)); + } + }, keys); + } + finally { + closeCassandraSession(ses); + } + } + + @Override public void write(final Cache.Entry entry) throws CacheWriterException { + if (entry == null || entry.getKey() == null) + return; + + CassandraSession ses = getCassandraSession(); + + try { + ses.execute(new ExecutionAssistant() { + @Override public boolean tableExistenceRequired() { + return true; + } + + @Override public String getStatement() { + return controller.getWriteStatement(); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement) { + return controller.bindKeyValue(statement, entry.getKey(), entry.getValue()); + } + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "WRITE"; + } + + @Override public Void process(Row row) { + return null; + } + }); + } + finally { + closeCassandraSession(ses); + } + } + + @Override public void writeAll(Collection> entries) throws CacheWriterException { + if (entries == null || entries.isEmpty()) + return; + + CassandraSession ses = getCassandraSession(); + + try { + ses.execute(new GenericBatchExecutionAssistant>() { + @Override public String getStatement() { + return controller.getWriteStatement(); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement, + Cache.Entry entry) { + return controller.bindKeyValue(statement, entry.getKey(), entry.getValue()); + } + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "BULK_WRITE"; + } + + @Override public boolean tableExistenceRequired() { + return true; + } + }, entries); + } + finally { + closeCassandraSession(ses); + } + } + + @Override public void delete(final Object key) throws CacheWriterException { + if (key == null) + return; + + CassandraSession ses = getCassandraSession(); + + try { + ses.execute(new ExecutionAssistant() { + @Override public boolean tableExistenceRequired() { + return false; + } + + @Override public String getStatement() { + return controller.getDeleteStatement(); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement) { + return controller.bindKey(statement, key); + } + + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "DELETE"; + } + + @Override public Void process(Row row) { + return null; + } + }); + } + finally { + closeCassandraSession(ses); + } + } + + @Override public void deleteAll(Collection keys) throws CacheWriterException { + if (keys == null || keys.isEmpty()) + return; + + CassandraSession ses = getCassandraSession(); + + try { + ses.execute(new GenericBatchExecutionAssistant() { + @Override public String getStatement() { + return controller.getDeleteStatement(); + } + + @Override public BoundStatement bindStatement(PreparedStatement statement, Object key) { + return controller.bindKey(statement, key); + } + + @Override public KeyValuePersistenceSettings getPersistenceSettings() { + return controller.getPersistenceSettings(); + } + + @Override public String operationName() { + return "BULK_DELETE"; + } + }, keys); + } + finally { + closeCassandraSession(ses); + } + } + + private CassandraSession getCassandraSession() { + if (storeSession == null || storeSession.transaction() == null) + return dataSource.session(logger != null ? logger : new NullLogger()); + + CassandraSession ses = (CassandraSession)storeSession.properties().get(ATTR_CONN_PROP); + + if (ses == null) { + ses = dataSource.session(logger != null ? logger : new NullLogger()); + storeSession.properties().put(ATTR_CONN_PROP, ses); + } + + return ses; + } + + private void closeCassandraSession(CassandraSession ses) { + if (ses != null && (storeSession == null || storeSession.transaction() == null)) { + try { + ses.close(); + } + catch (Throwable ignored) { + } + } + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java new file mode 100644 index 0000000..2497d92 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStoreFactory.java @@ -0,0 +1,123 @@ +/* + * 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; + +import javax.cache.configuration.Factory; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.cassandra.utils.datasource.DataSource; +import org.apache.ignite.cache.store.cassandra.utils.persistence.KeyValuePersistenceSettings; +import org.apache.ignite.internal.IgniteComponentType; +import org.apache.ignite.internal.util.spring.IgniteSpringHelper; +import org.apache.ignite.resources.SpringApplicationContextResource; + +/** + * Factory class to instantiate ${@link org.apache.ignite.cache.store.cassandra.CassandraCacheStore} + * @param Ignite cache key type + * @param Ignite cache value type + */ +public class CassandraCacheStoreFactory implements Factory> { + @SpringApplicationContextResource + private Object appContext; + + private String dataSourceBean; + private String persistenceSettingsBean; + + private transient DataSource dataSource; + private transient KeyValuePersistenceSettings persistenceSettings; + + @Override public CassandraCacheStore create() { + return new CassandraCacheStore<>(getDataSource(), getPersistenceSettings()); + } + + @SuppressWarnings("UnusedDeclaration") + public CassandraCacheStoreFactory setDataSource(DataSource dataSource) { + this.dataSource = dataSource; + return this; + } + + public CassandraCacheStoreFactory setDataSourceBean(String bean) { + this.dataSourceBean = bean; + return this; + } + + @SuppressWarnings("UnusedDeclaration") + public CassandraCacheStoreFactory setPersistenceSettings(KeyValuePersistenceSettings settings) { + this.persistenceSettings = settings; + return this; + } + + public CassandraCacheStoreFactory setPersistenceSettingsBean(String bean) { + this.persistenceSettingsBean = bean; + return this; + } + + private DataSource getDataSource() { + if (dataSource != null) + return dataSource; + + if (dataSourceBean == null) + throw new IllegalStateException("Either DataSource bean or DataSource itself should be specified"); + + if (appContext == null) { + throw new IllegalStateException("Can't get Cassandra DataSource cause Spring application " + + "context wasn't injected into CassandraCacheStoreFactory"); + } + + Object obj = loadSpringContextBean(appContext, dataSourceBean); + + if (!(obj instanceof DataSource)) + throw new IllegalStateException("Incorrect connection bean '" + dataSourceBean + "' specified"); + + return dataSource = (DataSource)obj; + } + + private KeyValuePersistenceSettings getPersistenceSettings() { + if (persistenceSettings != null) + return persistenceSettings; + + if (persistenceSettingsBean == null) { + throw new IllegalStateException("Either persistence settings bean or persistence settings itself " + + "should be specified"); + } + + if (appContext == null) { + throw new IllegalStateException("Can't get Cassandra persistence settings cause Spring application " + + "context wasn't injected into CassandraCacheStoreFactory"); + } + + Object obj = loadSpringContextBean(appContext, persistenceSettingsBean); + + if (!(obj instanceof KeyValuePersistenceSettings)) { + throw new IllegalStateException("Incorrect persistence settings bean '" + + persistenceSettingsBean + "' specified"); + } + + return persistenceSettings = (KeyValuePersistenceSettings)obj; + } + + private Object loadSpringContextBean(Object appContext, String beanName) { + try { + IgniteSpringHelper spring = IgniteComponentType.SPRING.create(false); + return spring.loadBeanFromAppContext(appContext, beanName); + } + catch (Exception e) { + throw new IgniteException("Failed to load bean in application context [beanName=" + beanName + ", igniteConfig=" + appContext + ']', e); + } + } + +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/CassandraHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/CassandraHelper.java new file mode 100644 index 0000000..4d20eba --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/CassandraHelper.java @@ -0,0 +1,111 @@ +/* + * 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.common; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.exceptions.InvalidQueryException; +import com.datastax.driver.core.exceptions.NoHostAvailableException; +import com.datastax.driver.core.exceptions.ReadTimeoutException; +import java.util.regex.Pattern; + +/** + * Helper class providing methods to work with Cassandra session and exceptions + */ +public class CassandraHelper { + private static final Pattern KEYSPACE_EXIST_ERROR1 = Pattern.compile("Keyspace [0-9a-zA-Z_]+ does not exist"); + private static final Pattern KEYSPACE_EXIST_ERROR2 = Pattern.compile("Cannot add table '[0-9a-zA-Z_]+' to non existing keyspace.*"); + private static final Pattern TABLE_EXIST_ERROR = Pattern.compile("unconfigured table [0-9a-zA-Z_]+"); + + private static final String PREP_STATEMENT_CLUSTER_INSTANCE_ERROR = "You may have used a PreparedStatement that " + + "was created with another Cluster instance"; + + public static void closeSession(Session driverSession) { + if (driverSession == null) + return; + + Cluster cluster = driverSession.getCluster(); + + try { + if (!driverSession.isClosed()) + driverSession.close(); + } + catch (Throwable ignored) { + } + + try { + if (!cluster.isClosed()) + cluster.close(); + } + catch (Throwable ignored) { + } + } + + public static boolean isKeyspaceAbsenceError(Throwable e) { + while (e != null) { + if (e instanceof InvalidQueryException && + (KEYSPACE_EXIST_ERROR1.matcher(e.getMessage()).matches() || + KEYSPACE_EXIST_ERROR2.matcher(e.getMessage()).matches())) { + return true; + } + + e = e.getCause(); + } + + return false; + } + + public static boolean isTableAbsenceError(Throwable e) { + while (e != null) { + if (e instanceof InvalidQueryException && + (TABLE_EXIST_ERROR.matcher(e.getMessage()).matches() || + KEYSPACE_EXIST_ERROR1.matcher(e.getMessage()).matches() || + KEYSPACE_EXIST_ERROR2.matcher(e.getMessage()).matches())) { + return true; + } + + e = e.getCause(); + } + + return false; + } + + public static boolean isHostsAvailabilityError(Throwable e) { + while (e != null) { + if (e instanceof NoHostAvailableException || + e instanceof ReadTimeoutException) + return true; + + e = e.getCause(); + } + + return false; + } + + public static boolean isPreparedStatementClusterError(Throwable e) { + while (e != null) { + if (e instanceof InvalidQueryException && e.getMessage().contains(PREP_STATEMENT_CLUSTER_INSTANCE_ERROR)) + return true; + + e = e.getCause(); + } + + return false; + } +} + diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/PropertyMappingHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/PropertyMappingHelper.java new file mode 100644 index 0000000..565b506 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/PropertyMappingHelper.java @@ -0,0 +1,158 @@ +/* + * 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.common; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Row; +import java.beans.PropertyDescriptor; +import java.lang.annotation.Annotation; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.commons.beanutils.PropertyUtils; +import org.apache.ignite.cache.store.cassandra.utils.serializer.Serializer; + +/** + * Helper class providing bunch of methods to discover fields of POJO objects and + * map primitive Java types to appropriate Cassandra types + */ +public class PropertyMappingHelper { + private static final Class BYTES_ARRAY_CLASS = (new byte[] {}).getClass(); + + private static final Map JAVA_TO_CASSANDRA_MAPPING = new HashMap() {{ + put(String.class, DataType.Name.TEXT); + put(Integer.class, DataType.Name.INT); + put(int.class, DataType.Name.INT); + put(Short.class, DataType.Name.INT); + put(short.class, DataType.Name.INT); + put(Long.class, DataType.Name.BIGINT); + put(long.class, DataType.Name.BIGINT); + put(Double.class, DataType.Name.DOUBLE); + put(double.class, DataType.Name.DOUBLE); + put(Boolean.class, DataType.Name.BOOLEAN); + put(boolean.class, DataType.Name.BOOLEAN); + put(Float.class, DataType.Name.FLOAT); + put(float.class, DataType.Name.FLOAT); + put(ByteBuffer.class, DataType.Name.BLOB); + put(BYTES_ARRAY_CLASS, DataType.Name.BLOB); + put(BigDecimal.class, DataType.Name.DECIMAL); + put(InetAddress.class, DataType.Name.INET); + put(Date.class, DataType.Name.TIMESTAMP); + put(UUID.class, DataType.Name.UUID); + put(BigInteger.class, DataType.Name.VARINT); + }}; + + public static DataType.Name getCassandraType(Class clazz) + { + return JAVA_TO_CASSANDRA_MAPPING.get(clazz); + } + + public static PropertyDescriptor getPojoPropertyDescriptor(Class clazz, String property) { + List descriptors = getPojoPropertyDescriptors(clazz, false); + + if (descriptors == null || descriptors.isEmpty()) + throw new IllegalArgumentException("POJO class doesn't have '" + property + "' property"); + + for (PropertyDescriptor descriptor : descriptors) { + if (descriptor.getName().equals(property)) + return descriptor; + } + + throw new IllegalArgumentException("POJO class doesn't have '" + property + "' property"); + } + + public static List getPojoPropertyDescriptors(Class clazz, boolean primitive) { + return getPojoPropertyDescriptors(clazz, null, primitive); + } + + public static List getPojoPropertyDescriptors(Class clazz, + Class annotation, boolean primitive) { + PropertyDescriptor[] descriptors = PropertyUtils.getPropertyDescriptors(clazz); + + List list = new ArrayList<>(descriptors == null ? 1 : descriptors.length); + + if (descriptors == null || descriptors.length == 0) + return list; + + for (PropertyDescriptor descriptor : descriptors) { + if (descriptor.getReadMethod() == null || descriptor.getWriteMethod() == null || + (primitive && !isPrimitivePropertyDescriptor(descriptor))) { + continue; + } + + if (annotation == null || descriptor.getReadMethod().getAnnotation(annotation) != null) + list.add(descriptor); + } + + return list; + } + + public static boolean isPrimitivePropertyDescriptor(PropertyDescriptor descriptor) + { + return PropertyMappingHelper.JAVA_TO_CASSANDRA_MAPPING.containsKey(descriptor.getPropertyType()); + } + + public static Object getCassandraColumnValue(Row row, String column, Class clazz, Serializer serializer) { + if (String.class.equals(clazz)) + return row.getString(column); + else if (Integer.class.equals(clazz) || int.class.equals(clazz)) + return row.getInt(column); + else if (Short.class.equals(clazz) || short.class.equals(clazz)) + return (short)row.getInt(column); + else if (Long.class.equals(clazz) || long.class.equals(clazz)) + return row.getLong(column); + else if (Double.class.equals(clazz) || double.class.equals(clazz)) + return row.getDouble(column); + else if (Boolean.class.equals(clazz) || boolean.class.equals(clazz)) + return row.getBool(column); + else if (Float.class.equals(clazz) || float.class.equals(clazz)) + return row.getFloat(column); + else if (ByteBuffer.class.equals(clazz)) + return row.getBytes(column); + else if (PropertyMappingHelper.BYTES_ARRAY_CLASS.equals(clazz)) { + ByteBuffer buffer = row.getBytes(column); + return buffer == null ? null : buffer.array(); + } + else if (BigDecimal.class.equals(clazz)) + return row.getDecimal(column); + else if (InetAddress.class.equals(clazz)) + return row.getInet(column); + else if (Date.class.equals(clazz)) + return row.getDate(column); + else if (UUID.class.equals(clazz)) + return row.getUUID(column); + else if (BigInteger.class.equals(clazz)) + return row.getVarint(column); + + if (serializer == null) { + throw new IllegalStateException("Can't deserialize value from '" + column + "' Cassandra column, " + + "cause there is no BLOB serializer specified"); + } + + ByteBuffer buffer = row.getBytes(column); + + return buffer == null ? null : serializer.deserialize(buffer); + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/SystemHelper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/SystemHelper.java new file mode 100644 index 0000000..a183916 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/common/SystemHelper.java @@ -0,0 +1,48 @@ +/* + * 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.common; + +import java.net.InetAddress; +import java.net.UnknownHostException; + +/** + * Helper class providing system information about the host (ip, hostname, os and etc.) + */ +public class SystemHelper { + public static final int PROCESSORS_COUNT = Runtime.getRuntime().availableProcessors(); + public static final String LINE_SEPARATOR = System.getProperty("line.separator"); + public static final String DOUBLE_LINE_SEPARATOR = LINE_SEPARATOR + LINE_SEPARATOR; + public static final String OS_NAME = System.getProperty("os.name"); + public static final String OS_USER = System.getProperty("user.name"); + public static final boolean IS_WINDOWS = OS_NAME.toLowerCase().contains("win"); + public static final boolean IS_MAC = OS_NAME.toLowerCase().contains("mac"); + public static final boolean IS_UNIX = OS_NAME.toLowerCase().contains("nux"); + public static final String HOST_NAME; + public static final String HOST_IP; + + static { + try { + InetAddress address = InetAddress.getLocalHost(); + HOST_NAME = address.getHostName(); + HOST_IP = address.getHostAddress(); + } + catch (UnknownHostException e) { + throw new IllegalStateException("Failed to get host/ip of current computer", e); + } + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/Credentials.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/Credentials.java new file mode 100644 index 0000000..6a6391d --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/Credentials.java @@ -0,0 +1,27 @@ +/* + * 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.datasource; + +/** + * Provides users/password credentials for Cassandra (instead of specifying user/password directly in Spring context xml) + */ +public interface Credentials { + public String getUser(); + + public String getPassword(); +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/DataSource.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/DataSource.java new file mode 100644 index 0000000..fb75c7f --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/DataSource.java @@ -0,0 +1,348 @@ +/* + * 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.datasource; + +import com.datastax.driver.core.AuthProvider; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.ConsistencyLevel; +import com.datastax.driver.core.NettyOptions; +import com.datastax.driver.core.PoolingOptions; +import com.datastax.driver.core.ProtocolOptions; +import com.datastax.driver.core.SSLOptions; +import com.datastax.driver.core.SocketOptions; +import com.datastax.driver.core.policies.AddressTranslater; +import com.datastax.driver.core.policies.LoadBalancingPolicy; +import com.datastax.driver.core.policies.ReconnectionPolicy; +import com.datastax.driver.core.policies.RetryPolicy; +import com.datastax.driver.core.policies.SpeculativeExecutionPolicy; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.store.cassandra.utils.session.CassandraSession; +import org.apache.ignite.cache.store.cassandra.utils.session.CassandraSessionImpl; + +/** + * Data source abstraction to specify configuration of the Cassandra session to be used + */ +public class DataSource { + private Integer fetchSize; + private ConsistencyLevel readConsistency; + private ConsistencyLevel writeConsistency; + + private String user; + private String password; + private Integer port; + private List contactPoints; + private List contactPointsWithPorts; + private Integer maxSchemaAgreementWaitSeconds; + private Integer protocolVersion; + private String compression; + private Boolean useSSL; + private Boolean collectMetrix; + private Boolean jmxReporting; + + private Credentials credentials; + private LoadBalancingPolicy loadBalancingPolicy; + private ReconnectionPolicy reconnectionPolicy; + private RetryPolicy retryPolicy; + private AddressTranslater addressTranslater; + private SpeculativeExecutionPolicy speculativeExecutionPolicy; + private AuthProvider authProvider; + private SSLOptions sslOptions; + private PoolingOptions poolingOptions; + private SocketOptions socketOptions; + private NettyOptions nettyOptions; + + private volatile CassandraSession session; + + @SuppressWarnings("UnusedDeclaration") + public void setUser(String user) { + this.user = user; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setPassword(String password) { + this.password = password; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setPort(int port) { + this.port = port; + invalidate(); + } + + 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(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setMaxSchemaAgreementWaitSeconds(int seconds) { + maxSchemaAgreementWaitSeconds = seconds; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setProtocolVersion(int version) { + protocolVersion = version; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + 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(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setUseSSL(boolean use) { + useSSL = use; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setCollectMetrix(boolean collect) { + collectMetrix = collect; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setJmxReporting(boolean enableReporting) { + jmxReporting = enableReporting; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setFetchSize(int size) { + fetchSize = size; + invalidate(); + } + + public void setReadConsistency(String level) { + readConsistency = parseConsistencyLevel(level); + invalidate(); + } + + public void setWriteConsistency(String level) { + writeConsistency = parseConsistencyLevel(level); + invalidate(); + } + + public void setCredentials(Credentials credentials) { + this.credentials = credentials; + invalidate(); + } + + public void setLoadBalancingPolicy(LoadBalancingPolicy policy) { + this.loadBalancingPolicy = policy; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setReconnectionPolicy(ReconnectionPolicy policy) { + this.reconnectionPolicy = policy; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setRetryPolicy(RetryPolicy policy) { + this.retryPolicy = policy; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setAddressTranslater(AddressTranslater translater) { + this.addressTranslater = translater; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setSpeculativeExecutionPolicy(SpeculativeExecutionPolicy policy) { + this.speculativeExecutionPolicy = policy; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setAuthProvider(AuthProvider provider) { + this.authProvider = provider; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setSslOptions(SSLOptions options) { + this.sslOptions = options; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setPoolingOptions(PoolingOptions options) { + this.poolingOptions = options; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setSocketOptions(SocketOptions options) { + this.socketOptions = options; + invalidate(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setNettyOptions(NettyOptions options) { + this.nettyOptions = options; + invalidate(); + } + + @SuppressWarnings("deprecation") + public synchronized CassandraSession session(IgniteLogger logger) { + if (session != null) + return session; + + Cluster.Builder builder = Cluster.builder(); + + if (user != null) + builder = builder.withCredentials(user, password); + + 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 (protocolVersion != null) + builder = builder.withProtocolVersion(protocolVersion); + + 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 (credentials != null) + builder = builder.withCredentials(credentials.getUser(), credentials.getPassword()); + + if (loadBalancingPolicy != null) + builder = builder.withLoadBalancingPolicy(loadBalancingPolicy); + + if (reconnectionPolicy != null) + builder = builder.withReconnectionPolicy(reconnectionPolicy); + + if (retryPolicy != null) + builder = builder.withRetryPolicy(retryPolicy); + + if (addressTranslater != null) + builder = builder.withAddressTranslater(addressTranslater); + + if (speculativeExecutionPolicy != null) + builder = builder.withSpeculativeExecutionPolicy(speculativeExecutionPolicy); + + if (authProvider != null) + builder = builder.withAuthProvider(authProvider); + + if (poolingOptions != null) + builder = builder.withPoolingOptions(poolingOptions); + + if (socketOptions != null) + builder = builder.withSocketOptions(socketOptions); + + if (nettyOptions != null) + builder = builder.withNettyOptions(nettyOptions); + + return session = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, logger); + } + + 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); + } + } + + private synchronized void invalidate() { + session = null; + } + +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/PlainCredentials.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/PlainCredentials.java new file mode 100644 index 0000000..dda1b37 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/datasource/PlainCredentials.java @@ -0,0 +1,40 @@ +/* + * 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.datasource; + +/** + * Simple implementation of ${@link org.apache.ignite.cache.store.cassandra.utils.datasource.Credentials} which + * just uses its constructor to hold user/password values + */ +public class PlainCredentials implements Credentials { + private String user; + private String password; + + public PlainCredentials(String user, String password) { + this.user = user; + this.password = password; + } + + @Override public String getUser() { + return user; + } + + @Override public String getPassword() { + return password; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyPersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyPersistenceSettings.java new file mode 100644 index 0000000..dbfc0ae --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyPersistenceSettings.java @@ -0,0 +1,223 @@ +/* + * 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.persistence; + +import java.beans.PropertyDescriptor; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.cache.affinity.AffinityKeyMapped; +import org.apache.ignite.cache.store.cassandra.utils.common.PropertyMappingHelper; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; + +/** + * Stores persistence settings for Ignite cache key + */ +public class KeyPersistenceSettings extends PersistenceSettings { + private static final String PARTITION_KEY_ELEMENT = "partitionKey"; + private static final String CLUSTER_KEY_ELEMENT = "clusterKey"; + private static final String FIELD_ELEMENT = "field"; + + private List fields = new LinkedList<>(); + private List partitionKeyFields = new LinkedList<>(); + private List clusterKeyFields = new LinkedList<>(); + + public KeyPersistenceSettings(Element el) { + super(el); + + if (!PersistenceStrategy.POJO.equals(getStrategy())) + return; + + Element partitionKeysNode = el.getElementsByTagName(PARTITION_KEY_ELEMENT) != null ? + (Element)el.getElementsByTagName(PARTITION_KEY_ELEMENT).item(0) : null; + + Element clusterKeysNode = el.getElementsByTagName(CLUSTER_KEY_ELEMENT) != null ? + (Element)el.getElementsByTagName(CLUSTER_KEY_ELEMENT).item(0) : null; + + if (partitionKeysNode == null && clusterKeysNode != null) { + throw new IllegalArgumentException("It's not allowed to specify cluster key fields mapping, but " + + "doesn't specify partition key mappings"); + } + + partitionKeyFields = detectFields(partitionKeysNode, getPartitionKeyDescriptors()); + + if (partitionKeyFields == null || partitionKeyFields.isEmpty()) { + throw new IllegalStateException("Failed to initialize partition key fields for class '" + + getJavaClass().getName() + "'"); + } + + clusterKeyFields = detectFields(clusterKeysNode, getClusterKeyDescriptors(partitionKeyFields)); + + fields = new LinkedList<>(); + fields.addAll(partitionKeyFields); + fields.addAll(clusterKeyFields); + + checkDuplicates(fields); + } + + @Override public List getFields() { + return fields; + } + + public String getPrimaryKeyDDL() { + StringBuilder partitionKey = new StringBuilder(); + + List columns = getPartitionKeyColumns(); + for (String column : columns) { + if (partitionKey.length() != 0) + partitionKey.append(", "); + + partitionKey.append(column); + } + + StringBuilder clusterKey = new StringBuilder(); + + columns = getClusterKeyColumns(); + if (columns != null) { + for (String column : columns) { + if (clusterKey.length() != 0) + clusterKey.append(", "); + + clusterKey.append(column); + } + } + + return clusterKey.length() == 0 ? + "primary key ((" + partitionKey.toString() + "))" : + "primary key ((" + partitionKey.toString() + "), " + clusterKey.toString() + ")"; + } + + 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.equals(sortOrder); + + builder.append(field.getColumn()).append(" ").append(asc ? "asc" : "desc"); + } + + return builder.length() == 0 ? null : "clustering order by (" + builder.toString() + ")"; + } + + @Override protected String defaultColumnName() { + return "key"; + } + + private List getPartitionKeyColumns() { + List columns = new LinkedList<>(); + + if (PersistenceStrategy.BLOB.equals(getStrategy()) || PersistenceStrategy.PRIMITIVE.equals(getStrategy())) { + columns.add(getColumn()); + return columns; + } + + if (partitionKeyFields != null) { + for (PojoField field : partitionKeyFields) + columns.add(field.getColumn()); + } + + return columns; + } + + private List getClusterKeyColumns() { + List columns = new LinkedList<>(); + + if (clusterKeyFields != null) { + for (PojoField field : clusterKeyFields) + columns.add(field.getColumn()); + } + + return columns; + } + + private List detectFields(Element el, List descriptors) { + List list = new LinkedList<>(); + + if (el == null && (descriptors == null || descriptors.isEmpty())) + return list; + + if (el == null) { + for (PropertyDescriptor descriptor : descriptors) + list.add(new PojoKeyField(descriptor)); + + return list; + } + + NodeList nodes = el.getElementsByTagName(FIELD_ELEMENT); + int count = nodes == null ? 0 : nodes.getLength(); + + if (count == 0) { + throw new IllegalArgumentException("Incorrect configuration of Cassandra key persistence settings, " + + "no cluster key fields specified inside '" + PARTITION_KEY_ELEMENT + "/" + + CLUSTER_KEY_ELEMENT + "' element"); + } + + for (int i = 0; i < count; i++) { + PojoKeyField field = new PojoKeyField((Element)nodes.item(i), getJavaClass()); + + PropertyDescriptor descriptor = findPropertyDescriptor(descriptors, field.getName()); + + if (descriptor == null) { + throw new IllegalArgumentException("Specified POJO field '" + field.getName() + + "' doesn't exist in '" + getJavaClass().getName() + "' class"); + } + + list.add(field); + } + + return list; + } + + private List getPartitionKeyDescriptors() { + List primitivePropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), + AffinityKeyMapped.class, true); + + return primitivePropDescriptors != null && !primitivePropDescriptors.isEmpty() ? + primitivePropDescriptors : + PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true); + } + + private List getClusterKeyDescriptors(List partitionKeyFields) { + List primitivePropDescriptors = + PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true); + + if (primitivePropDescriptors == null || primitivePropDescriptors.isEmpty() || + partitionKeyFields.size() == primitivePropDescriptors.size()) { + return null; + } + + for (PojoField field : partitionKeyFields) { + for (int i = 0; i < primitivePropDescriptors.size(); i++) { + if (primitivePropDescriptors.get(i).getName().equals(field.getName())) { + primitivePropDescriptors.remove(i); + break; + } + } + } + + return primitivePropDescriptors; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyValuePersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyValuePersistenceSettings.java new file mode 100644 index 0000000..67de888 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/KeyValuePersistenceSettings.java @@ -0,0 +1,356 @@ +/* + * 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.persistence; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.StringReader; +import java.util.LinkedList; +import java.util.List; +import javax.xml.parsers.DocumentBuilder; +import javax.xml.parsers.DocumentBuilderFactory; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.cassandra.utils.common.SystemHelper; +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 { + private static final String KEYSPACE_ATTR = "keyspace"; + private static final String TABLE_ATTR = "table"; + private static final String LOAD_ON_STARTUP_ATTR = "loadOnStartup"; + private static final String TTL_ATTR = "ttl"; + private static final String PERSISTENCE_NODE = "persistence"; + private static final String KEYSPACE_OPTIONS_NODE = "keyspaceOptions"; + private static final String TABLE_OPTIONS_NODE = "tableOptions"; + private static final String KEY_PERSISTENCE_NODE = "keyPersistence"; + private static final String VALUE_PERSISTENCE_NODE = "valuePersistence"; + + private int startupLoadingCount = 10000; + private Integer ttl; + private String keyspace; + private String table; + private String tableOptions; + private String keyspaceOptions = "replication = {'class' : 'SimpleStrategy', 'replication_factor' : 3} " + + "and durable_writes = true"; + + private KeyPersistenceSettings keyPersistenceSettings; + private ValuePersistenceSettings valuePersistenceSettings; + + @SuppressWarnings("UnusedDeclaration") + public KeyValuePersistenceSettings(String settings) { + init(settings); + } + + public KeyValuePersistenceSettings(Resource settingsResource) { + init(loadSettings(settingsResource)); + } + + public int getStartupLoadingCount() { + return startupLoadingCount; + } + + public Integer getTTL() { + return ttl; + } + + public String getKeyspace() { + return keyspace; + } + + public String getTable() { + return table; + } + + public String getTableFullName() + { + return keyspace + "." + table; + } + + public KeyPersistenceSettings getKeyPersistenceSettings() { + return keyPersistenceSettings; + } + + public ValuePersistenceSettings getValuePersistenceSettings() { + return valuePersistenceSettings; + } + + @SuppressWarnings("UnusedDeclaration") + public List getFields() { + List fields = new LinkedList<>(); + + for (PojoField field : keyPersistenceSettings.getFields()) + fields.add(field); + + for (PojoField field : valuePersistenceSettings.getFields()) + fields.add(field); + + return fields; + } + + @SuppressWarnings("UnusedDeclaration") + public List getKeyFields() { + return keyPersistenceSettings.getFields(); + } + + @SuppressWarnings("UnusedDeclaration") + public List getValueFields() { + return valuePersistenceSettings.getFields(); + } + + public String getKeyspaceDDLStatement() { + StringBuilder builder = new StringBuilder(); + builder.append("create keyspace if not exists ").append(keyspace); + + if (keyspaceOptions != null) { + if (!keyspaceOptions.trim().toLowerCase().startsWith("with")) + builder.append(" with"); + + builder.append(" ").append(keyspaceOptions); + } + + String statement = builder.toString().trim(); + + if (!statement.endsWith(";")) + statement += ";"; + + return statement; + } + + public String getTableDDLStatement() { + String columnsDDL = keyPersistenceSettings.getTableColumnsDDL() + ", " + valuePersistenceSettings.getTableColumnsDDL(); + + String primaryKeyDDL = keyPersistenceSettings.getPrimaryKeyDDL(); + String clusteringDDL = keyPersistenceSettings.getClusteringDDL(); + + String optionsDDL = tableOptions != null && !tableOptions.trim().isEmpty() ? tableOptions.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); + builder.append(" (").append(columnsDDL).append(", ").append(primaryKeyDDL).append(")"); + + if (!optionsDDL.isEmpty()) + builder.append(" ").append(optionsDDL); + + String tableDDL = builder.toString().trim(); + + return tableDDL.endsWith(";") ? tableDDL : tableDDL + ";"; + } + + public List getIndexDDLStatements() { + List indexDDLs = new LinkedList<>(); + + List fields = valuePersistenceSettings.getFields(); + + for (PojoField field : fields) { + if (((PojoValueField)field).isIndexed()) + indexDDLs.add(((PojoValueField)field).getIndexDDL(keyspace, table)); + } + + return indexDDLs; + } + + private String loadSettings(Resource resource) { + StringBuilder settings = new StringBuilder(); + InputStream in; + BufferedReader reader = null; + + try { + in = resource.getInputStream(); + } + catch (IOException e) { + throw new IgniteException("Failed to get input stream for Cassandra persistence settings resource: " + resource, e); + } + + 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 resource: " + resource, e); + } + finally { + if (reader != null) { + try { + reader.close(); + } + catch (Throwable ignored) { + } + } + + if (in != null) { + try { + in.close(); + } + catch (Throwable ignored) { + } + } + } + + return settings.toString(); + } + + @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"); + } + + if (!root.hasAttribute(TABLE_ATTR)) { + throw new IllegalArgumentException("Incorrect persistence settings '" + TABLE_ATTR + "' attribute " + + "should be specified"); + } + + keyspace = root.getAttribute(KEYSPACE_ATTR).trim(); + table = root.getAttribute(TABLE_ATTR).trim(); + + if (root.hasAttribute(LOAD_ON_STARTUP_ATTR)) { + String val = root.getAttribute(LOAD_ON_STARTUP_ATTR).trim(); + + try { + startupLoadingCount = Integer.parseInt(val); + } + catch (NumberFormatException e) { + throw new IllegalArgumentException("Incorrect value '" + val + "' specified for '" + LOAD_ON_STARTUP_ATTR + "' attribute"); + } + } + + if (root.hasAttribute(TTL_ATTR)) { + String val = root.getAttribute(TTL_ATTR).trim(); + + try { + ttl = Integer.parseInt(val); + } + catch (NumberFormatException e) { + throw new IllegalArgumentException("Incorrect value '" + val + "' specified for '" + TTL_ATTR + "' attribute"); + } + } + + 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 count = children.getLength(); + + for (int i = 0; i < count; 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)) { + tableOptions = el.getTextContent(); + tableOptions = tableOptions.replace("\n", " ").replace("\r", ""); + } + else if (nodeName.equals(KEYSPACE_OPTIONS_NODE)) { + keyspaceOptions = el.getTextContent(); + keyspaceOptions = keyspaceOptions.replace("\n", " ").replace("\r", ""); + } + else if (nodeName.equals(KEY_PERSISTENCE_NODE)) + keyPersistenceSettings = new KeyPersistenceSettings(el); + else if (nodeName.equals(VALUE_PERSISTENCE_NODE)) + valuePersistenceSettings = new ValuePersistenceSettings(el); + } + + if (keyPersistenceSettings == null) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification," + + " there are no key persistence settings specified"); + } + + if (valuePersistenceSettings == null) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification," + + " there are no value persistence settings specified"); + } + + List keyFields = keyPersistenceSettings.getFields(); + List valueFields = valuePersistenceSettings.getFields(); + + if (PersistenceStrategy.POJO.equals(keyPersistenceSettings.getStrategy()) && + (keyFields == null || keyFields.isEmpty())) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification," + + " there are no key fields found"); + } + + if (PersistenceStrategy.POJO.equals(valuePersistenceSettings.getStrategy()) && + (valueFields == null || valueFields.isEmpty())) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification," + + " there are no value fields found"); + } + + if (keyFields == null || keyFields.isEmpty() || valueFields == null || valueFields.isEmpty()) + return; + + for (PojoField keyField : keyFields) { + for (PojoField valField : valueFields) { + if (keyField.getColumn().equals(valField.getColumn())) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings specification," + + " key column '" + keyField.getColumn() + "' also specified as a value column"); + } + } + } + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceController.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceController.java new file mode 100644 index 0000000..e8cd5d6 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceController.java @@ -0,0 +1,303 @@ +/* + * 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.persistence; + +import com.datastax.driver.core.BoundStatement; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.Row; +import java.nio.ByteBuffer; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.store.cassandra.utils.common.PropertyMappingHelper; +import org.apache.ignite.cache.store.cassandra.utils.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 { + private KeyValuePersistenceSettings persistenceSettings; + + private String writeStatement; + private String deleteStatement; + private String loadStatement; + private String loadStatementWithKeyFields; + + public PersistenceController(KeyValuePersistenceSettings settings) { + if (settings == null) + throw new IllegalArgumentException("Persistent settings can't be null"); + + this.persistenceSettings = settings; + } + + public KeyValuePersistenceSettings getPersistenceSettings() { + return persistenceSettings; + } + + public String getKeyspace() { + return persistenceSettings.getKeyspace(); + } + + public String getTable() { + return persistenceSettings.getTable(); + } + + public String getWriteStatement() { + if (writeStatement != null) { + return writeStatement; + } + + List columns = getKeyValueColumns(); + + StringBuilder columnsList = new StringBuilder(); + StringBuilder questionsList = new StringBuilder(); + + for (String column : columns) { + if (columnsList.length() != 0) { + columnsList.append(", "); + questionsList.append(","); + } + + columnsList.append(column); + questionsList.append("?"); + } + + writeStatement = "insert into " + persistenceSettings.getKeyspace() + "." + persistenceSettings.getTable() + " (" + + columnsList.toString() + ") values (" + questionsList.toString() + ")"; + + if (persistenceSettings.getTTL() != null) + writeStatement += " using ttl " + persistenceSettings.getTTL(); + + writeStatement += ";"; + + return writeStatement; + } + + public String getDeleteStatement() { + if (deleteStatement != null) { + return deleteStatement; + } + + List columns = getKeyColumns(); + + StringBuilder statement = new StringBuilder(); + + for (String column : columns) { + if (statement.length() != 0) + statement.append(" and "); + + statement.append(column).append("=?"); + } + + statement.append(";"); + + deleteStatement = "delete from " + + persistenceSettings.getKeyspace() + "." + + persistenceSettings.getTable() + " where " + + statement.toString(); + + return deleteStatement; + } + + public String getLoadStatement(boolean includeKeyFields) { + if (loadStatement != null && loadStatementWithKeyFields != null) + return includeKeyFields ? loadStatementWithKeyFields : loadStatement; + + List valColumns = getValueColumns(); + List keyColumns = getKeyColumns(); + + StringBuilder headerWithKeyFields = new StringBuilder("select "); + + for (int i = 0; i < keyColumns.size(); i++) { + if (i > 0) + headerWithKeyFields.append(", "); + + headerWithKeyFields.append(keyColumns.get(i)); + } + + StringBuilder header = new StringBuilder("select "); + + for (int i = 0; i < valColumns.size(); i++) { + if (i > 0) + header.append(", "); + + headerWithKeyFields.append(","); + + header.append(valColumns.get(i)); + headerWithKeyFields.append(valColumns.get(i)); + } + + StringBuilder statement = new StringBuilder(); + + statement.append(" from "); + statement.append(persistenceSettings.getKeyspace()); + statement.append(".").append(persistenceSettings.getTable()); + statement.append(" where "); + + for (int i = 0; i < keyColumns.size(); i++) { + if (i > 0) + statement.append(" and "); + + statement.append(keyColumns.get(i)).append("=?"); + } + + statement.append(";"); + + loadStatement = header.toString() + statement.toString(); + loadStatementWithKeyFields = headerWithKeyFields.toString() + statement.toString(); + + return includeKeyFields ? loadStatementWithKeyFields : loadStatement; + } + + public BoundStatement bindKey(PreparedStatement statement, Object key) { + KeyPersistenceSettings settings = persistenceSettings.getKeyPersistenceSettings(); + + Object[] values = getBindingValues(settings.getStrategy(), + settings.getSerializer(), settings.getFields(), key); + + return statement.bind(values); + } + + public BoundStatement bindKeyValue(PreparedStatement statement, Object key, Object value) { + KeyPersistenceSettings keySettings = persistenceSettings.getKeyPersistenceSettings(); + Object[] keyValues = getBindingValues(keySettings.getStrategy(), + keySettings.getSerializer(), keySettings.getFields(), key); + + ValuePersistenceSettings valSettings = persistenceSettings.getValuePersistenceSettings(); + Object[] valValues = getBindingValues(valSettings.getStrategy(), + valSettings.getSerializer(), valSettings.getFields(), value); + + Object[] values = new Object[keyValues.length + valValues.length]; + + int i = 0; + + for (Object val : keyValues) { + values[i] = val; + i++; + } + + for (Object val : valValues) { + values[i] = val; + i++; + } + + return statement.bind(values); + } + + @SuppressWarnings("UnusedDeclaration") + public Object buildKeyObject(Row row) { + return buildObject(row, persistenceSettings.getKeyPersistenceSettings()); + } + + public Object buildValueObject(Row row) { + return buildObject(row, persistenceSettings.getValuePersistenceSettings()); + } + + private Object buildObject(Row row, PersistenceSettings settings) { + if (row == null) + return null; + + PersistenceStrategy strategy = settings.getStrategy(); + Class clazz = settings.getJavaClass(); + String column = settings.getColumn(); + List fields = settings.getFields(); + + if (PersistenceStrategy.PRIMITIVE.equals(strategy)) + return PropertyMappingHelper.getCassandraColumnValue(row, column, clazz, null); + + if (PersistenceStrategy.BLOB.equals(strategy)) + return settings.getSerializer().deserialize(row.getBytes(column)); + + 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) + field.setValueFromRow(row, obj, settings.getSerializer()); + + return obj; + } + + private Object[] getBindingValues(PersistenceStrategy strategy, Serializer serializer, + List fields, Object obj) { + if (PersistenceStrategy.PRIMITIVE.equals(strategy)) { + 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."); + } + + return new Object[] {obj}; + } + + if (PersistenceStrategy.BLOB.equals(strategy)) + return new Object[] {serializer.serialize(obj)}; + + Object[] values = new Object[fields.size()]; + + int i = 0; + + for (PojoField field : fields) { + Object value = field.getValueFromObject(obj, serializer); + + if (value instanceof byte[]) + value = ByteBuffer.wrap((byte[])value); + + values[i] = value; + + i++; + } + + return values; + } + + private List getKeyValueColumns() { + List columns = getKeyColumns(); + columns.addAll(getValueColumns()); + return columns; + } + + private List getKeyColumns() { + return getColumns(persistenceSettings.getKeyPersistenceSettings()); + } + + private List getValueColumns() { + return getColumns(persistenceSettings.getValuePersistenceSettings()); + } + + private List getColumns(PersistenceSettings settings) { + List columns = new LinkedList<>(); + + if (!PersistenceStrategy.POJO.equals(settings.getStrategy())) { + columns.add(settings.getColumn()); + return columns; + } + + for (PojoField field : settings.getFields()) + columns.add(field.getColumn()); + + return columns; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceSettings.java new file mode 100644 index 0000000..9e7096b --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceSettings.java @@ -0,0 +1,252 @@ +/* + * 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.persistence; + +import com.datastax.driver.core.DataType; +import java.beans.PropertyDescriptor; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.ignite.cache.store.cassandra.utils.common.PropertyMappingHelper; +import org.apache.ignite.cache.store.cassandra.utils.serializer.JavaSerializer; +import org.apache.ignite.cache.store.cassandra.utils.serializer.Serializer; +import org.w3c.dom.Element; + +/** + * Stores persistence settings, which describes how particular key/value from Ignite cache + * should be stored in Cassandra + */ +public abstract class PersistenceSettings { + private static final String STRATEGY_ATTR = "strategy"; + private static final String COLUMN_ATTR = "column"; + private static final String SERIALIZER_ATTR = "serializer"; + private static final String CLASS_ATTR = "class"; + + private PersistenceStrategy strategy; + private Class javaClass; + private String column; + private Serializer serializer = new JavaSerializer(); + + public static PropertyDescriptor findPropertyDescriptor(List descriptors, String propertyName) { + if (descriptors == null || descriptors.isEmpty() || propertyName == null || propertyName.trim().isEmpty()) + return null; + + for (PropertyDescriptor descriptor : descriptors) { + if (descriptor.getName().equals(propertyName)) + return descriptor; + } + + return null; + } + + @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 { + strategy = PersistenceStrategy.valueOf(el.getAttribute(STRATEGY_ATTR).trim().toUpperCase()); + } + catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Incorrect persistence strategy specified: " + el.getAttribute(STRATEGY_ATTR)); + } + + if (!el.hasAttribute(CLASS_ATTR) && !PersistenceStrategy.BLOB.equals(strategy)) { + throw new IllegalArgumentException("DOM element representing key/value persistence object should have '" + + CLASS_ATTR + "' attribute or have BLOB persistence strategy"); + } + + try { + javaClass = 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.equals(strategy) && + (javaClass.equals(ByteBuffer.class) || javaClass.equals(byte[].class))) { + throw new IllegalArgumentException("Java class '" + el.getAttribute(CLASS_ATTR) + "' " + + "specified could only be persisted using BLOB persistence strategy"); + } + + if (PersistenceStrategy.PRIMITIVE.equals(strategy) && + PropertyMappingHelper.getCassandraType(javaClass) == null) { + throw new IllegalArgumentException("Current implementation doesn't support persisting '" + + javaClass.getName() + "' object using PRIMITIVE strategy"); + } + + if (PersistenceStrategy.POJO.equals(strategy)) { + try { + javaClass.getConstructor(); + } + catch (Throwable e) { + throw new IllegalArgumentException("Java class '" + javaClass.getName() + "' couldn't be used as POJO " + + "cause it doesn't have no arguments constructor", e); + } + } + + if (el.hasAttribute(COLUMN_ATTR)) { + if (!PersistenceStrategy.BLOB.equals(strategy) && !PersistenceStrategy.PRIMITIVE.equals(strategy)) { + throw new IllegalArgumentException("Incorrect configuration of Cassandra key/value persistence settings, " + + "'" + COLUMN_ATTR + "' attribute is only applicable for PRIMITIVE or BLOB strategy"); + } + + column = el.getAttribute(COLUMN_ATTR).trim(); + } + + if (el.hasAttribute(SERIALIZER_ATTR)) { + if (!PersistenceStrategy.BLOB.equals(strategy) && !PersistenceStrategy.POJO.equals(strategy)) { + 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.equals(strategy) || PersistenceStrategy.PRIMITIVE.equals(strategy)) && column == null) + column = defaultColumnName(); + } + + public Class getJavaClass() { + return javaClass; + } + + public PersistenceStrategy getStrategy() { + return strategy; + } + + public String getColumn() { + return column; + } + + public Serializer getSerializer() { + return serializer; + } + + public abstract List getFields(); + + public String getTableColumnsDDL() { + if (PersistenceStrategy.BLOB.equals(strategy)) { + return column + " " + DataType.Name.BLOB.toString(); + } + + if (PersistenceStrategy.PRIMITIVE.equals(strategy)) { + return column + " " + PropertyMappingHelper.getCassandraType(javaClass); + } + + StringBuilder builder = new StringBuilder(); + + for (PojoField field : getFields()) { + if (builder.length() > 0) + builder.append(", "); + + builder.append(field.getColumnDDL()); + } + + if (builder.length() == 0) { + throw new IllegalStateException("There are no POJO fields found for '" + javaClass.toString() + + "' class to be presented as a Cassandra primary key"); + } + + return builder.toString(); + } + + protected abstract String defaultColumnName(); + + protected void checkDuplicates(List fields) { + if (fields == null || fields.isEmpty()) + return; + + for (PojoField field1 : fields) { + boolean sameNames = false; + boolean sameColumns = false; + + for (PojoField field2 : fields) { + if (field1.getName().equals(field2.getName())) { + if (sameNames) { + throw new IllegalArgumentException("Incorrect Cassandra key persistence settings," + + " two POJO fields with the same name '" + field1.getName() + "' specified"); + } + + sameNames = true; + } + + if (field1.getColumn().equals(field2.getColumn())) { + if (sameColumns) { + throw new IllegalArgumentException("Incorrect Cassandra persistence settings," + + " two POJO fields with the same column '" + field1.getColumn() + "' specified"); + } + + sameColumns = true; + } + } + } + } + + 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 RuntimeException("Failed to load class '" + clazz + "' using reflection"); + } + + private Object newObjectInstance(String clazz) { + try { + return getClassInstance(clazz).newInstance(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to instantiate class '" + clazz + + "' using default constructor", e); + } + } + +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceStrategy.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceStrategy.java new file mode 100644 index 0000000..7e669ad --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PersistenceStrategy.java @@ -0,0 +1,61 @@ +/* + * 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.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: + * + * 1) Primitive java type (like Integer, String, Long and etc) which could be directly mapped to appropriate + * Cassandra type. + */ + 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: + * + * 1) org.apache.ignite.cache.store.cassandra.utils.serializer.JavaSerializer - uses standard Java serialization + * framework + * + * 2) org.apache.ignite.cache.store.cassandra.utils.serializer.TachyonSerializer - uses Tachyon 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: + * + * 1) 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/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoField.java new file mode 100644 index 0000000..0dcadca --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoField.java @@ -0,0 +1,151 @@ +/* + * 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.persistence; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Row; +import java.beans.PropertyDescriptor; +import org.apache.ignite.IgniteException; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.apache.ignite.cache.store.cassandra.utils.common.PropertyMappingHelper; +import org.apache.ignite.cache.store.cassandra.utils.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 { + private static final String NAME_ATTR = "name"; + private static final String COLUMN_ATTR = "column"; + + private String name; + private String column; + private String columnDDL; + private PropertyDescriptor descriptor; + + public PojoField(Element el, Class pojoClass) { + 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.column = el.hasAttribute(COLUMN_ATTR) ? el.getAttribute(COLUMN_ATTR).trim() : name.toLowerCase(); + + init(PropertyMappingHelper.getPojoPropertyDescriptor(pojoClass, name)); + } + + public PojoField(PropertyDescriptor descriptor) { + this.name = descriptor.getName(); + + QuerySqlField sqlField = descriptor.getReadMethod() != null ? + descriptor.getReadMethod().getAnnotation(QuerySqlField.class) : + descriptor.getWriteMethod() == null ? + null : + descriptor.getWriteMethod().getAnnotation(QuerySqlField.class); + + this.column = sqlField != null && sqlField.name() != null ? sqlField.name() : name.toLowerCase(); + + init(descriptor); + + if (sqlField != null) + init(sqlField); + } + + public String getName() { + return name; + } + + public String getColumn() { + return column; + } + + public String getColumnDDL() { + return columnDDL; + } + + public Object getValueFromObject(Object obj, Serializer serializer) { + try { + Object value = descriptor.getReadMethod().invoke(obj); + + if (value == null) + return null; + + DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(value.getClass()); + + if (cassandraType != null) + return value; + + if (serializer == null) { + throw new IllegalStateException("Can't serialize value from object '" + + value.getClass().getName() + "' field '" + name + "', cause there is no BLOB serializer specified"); + } + + return serializer.serialize(value); + } + catch (Throwable e) { + throw new IgniteException("Failed to get value of the field '" + descriptor.getName() + "' from the instance " + + " of '" + obj.getClass().toString() + "' class", e); + } + } + + public void setValueFromRow(Row row, Object obj, Serializer serializer) { + Object value = PropertyMappingHelper.getCassandraColumnValue(row, column, descriptor.getPropertyType(), serializer); + + try { + descriptor.getWriteMethod().invoke(obj, value); + } + catch (Throwable e) { + throw new IgniteException("Failed to set value of the field '" + descriptor.getName() + "' of the instance " + + " of '" + obj.getClass().toString() + "' class", e); + } + } + + protected void init(QuerySqlField sqlField) { + } + + protected void init(PropertyDescriptor descriptor) { + if (descriptor.getReadMethod() == null) { + throw new IllegalArgumentException("Field '" + descriptor.getName() + + "' of the class instance '" + descriptor.getPropertyType().getName() + + "' doesn't provide getter method"); + } + + if (descriptor.getWriteMethod() == null) { + throw new IllegalArgumentException("Field '" + descriptor.getName() + + "' of POJO object instance of the class '" + descriptor.getPropertyType().getName() + + "' doesn't provide write method"); + } + + if (!descriptor.getReadMethod().isAccessible()) + descriptor.getReadMethod().setAccessible(true); + + if (!descriptor.getWriteMethod().isAccessible()) + descriptor.getWriteMethod().setAccessible(true); + + DataType.Name cassandraType = PropertyMappingHelper.getCassandraType(descriptor.getPropertyType()); + cassandraType = cassandraType == null ? DataType.Name.BLOB : cassandraType; + + this.descriptor = descriptor; + this.columnDDL = column + " " + cassandraType.toString(); + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoKeyField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoKeyField.java new file mode 100644 index 0000000..9287f54 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoKeyField.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.cassandra.utils.persistence; + +import java.beans.PropertyDescriptor; +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 static enum SortOrder { + ASC, DESC; + } + + private static final String SORT_ATTR = "sort"; + + private SortOrder sortOrder = null; + + public PojoKeyField(Element el, Class pojoClass) { + super(el, pojoClass); + + if (el.hasAttribute(SORT_ATTR)) { + try { + sortOrder = SortOrder.valueOf(el.getAttribute(SORT_ATTR).trim().toUpperCase()); + } + catch (IllegalArgumentException e) { + throw new IllegalArgumentException("Incorrect sort order '" + el.getAttribute(SORT_ATTR) + "' specified"); + } + } + } + + public PojoKeyField(PropertyDescriptor descriptor) { + super(descriptor); + } + + public SortOrder getSortOrder() { + return sortOrder; + } + + protected void init(QuerySqlField sqlField) { + if (sqlField.descending()) + sortOrder = SortOrder.DESC; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoValueField.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoValueField.java new file mode 100644 index 0000000..899b89f --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/PojoValueField.java @@ -0,0 +1,104 @@ +/* + * 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.persistence; + +import java.beans.PropertyDescriptor; +import org.apache.ignite.cache.query.annotations.QuerySqlField; +import org.w3c.dom.Element; + +/** + * Descriptor for Ignite value POJO class + */ +public class PojoValueField extends PojoField { + private static final String STATIC_ATTR = "static"; + private static final String INDEX_ATTR = "index"; + private static final String INDEX_CLASS_ATTR = "indexClass"; + private static final String INDEX_OPTIONS_ATTR = "indexOptions"; + + private Boolean isIndexed; + private String indexClass; + private String indexOptions; + private Boolean isStatic; + + public PojoValueField(Element el, Class pojoClass) { + super(el, pojoClass); + + 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)) + indexClass = el.getAttribute(INDEX_CLASS_ATTR).trim(); + + if (el.hasAttribute(INDEX_OPTIONS_ATTR)) { + indexOptions = el.getAttribute(INDEX_OPTIONS_ATTR).trim(); + + if (!indexOptions.toLowerCase().startsWith("with")) { + indexOptions = indexOptions.toLowerCase().startsWith("options") ? + "with " + indexOptions : + "with options = " + indexOptions; + } + } + } + + public PojoValueField(PropertyDescriptor descriptor) { + super(descriptor); + } + + public String getColumnDDL() { + String columnDDL = super.getColumnDDL(); + + if (isStatic != null && isStatic) + columnDDL = columnDDL + " static"; + + return columnDDL; + } + + public boolean isIndexed() { + return isIndexed != null && isIndexed; + } + + public String getIndexDDL(String keyspace, String table) { + if (isIndexed == null || !isIndexed) + return null; + + StringBuilder builder = new StringBuilder(); + + if (indexClass != null) + builder.append("create custom index if not exists on ").append(keyspace).append(".").append(table); + else + builder.append("create index if not exists on ").append(keyspace).append(".").append(table); + + builder.append(" (").append(getColumn()).append(")"); + + if (indexClass != null) + builder.append(" using '").append(indexClass).append("'"); + + if (indexOptions != null) + builder.append(" ").append(indexOptions); + + return builder.append(";").toString(); + } + + protected void init(QuerySqlField sqlField) { + if (sqlField.index()) + isIndexed = true; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/ValuePersistenceSettings.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/ValuePersistenceSettings.java new file mode 100644 index 0000000..3198c35 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/persistence/ValuePersistenceSettings.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.utils.persistence; + +import java.beans.PropertyDescriptor; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.cache.store.cassandra.utils.common.PropertyMappingHelper; +import org.w3c.dom.Element; +import org.w3c.dom.NodeList; + +/** + * Stores persistence settings for Ignite cache value + */ +public class ValuePersistenceSettings extends PersistenceSettings { + private static final String FIELD_ELEMENT = "field"; + + private List fields = new LinkedList<>(); + + public ValuePersistenceSettings(Element el) { + super(el); + + if (!PersistenceStrategy.POJO.equals(getStrategy())) + return; + + NodeList nodes = el.getElementsByTagName(FIELD_ELEMENT); + + fields = detectFields(nodes); + + if (fields.isEmpty()) + throw new IllegalStateException("Failed to initialize value fields for class '" + getJavaClass().getName() + "'"); + + checkDuplicates(fields); + } + + public List getFields() { + return fields == null ? null : Collections.unmodifiableList(fields); + } + + @Override protected String defaultColumnName() { + return "value"; + } + + private List detectFields(NodeList fieldNodes) { + List list = new LinkedList<>(); + + if (fieldNodes == null || fieldNodes.getLength() == 0) { + List primitivePropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), true); + for (PropertyDescriptor descriptor : primitivePropDescriptors) { + list.add(new PojoValueField(descriptor)); + } + + return list; + } + + List allPropDescriptors = PropertyMappingHelper.getPojoPropertyDescriptors(getJavaClass(), false); + + int count = fieldNodes.getLength(); + + for (int i = 0; i < count; i++) { + PojoValueField field = new PojoValueField((Element)fieldNodes.item(i), getJavaClass()); + + PropertyDescriptor descriptor = findPropertyDescriptor(allPropDescriptors, field.getName()); + + if (descriptor == null) { + throw new IllegalArgumentException("Specified POJO field '" + field.getName() + + "' doesn't exist in '" + getJavaClass().getName() + "' class"); + } + + list.add(field); + } + + return list; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/JavaSerializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/JavaSerializer.java new file mode 100644 index 0000000..94ffd5d --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/JavaSerializer.java @@ -0,0 +1,99 @@ +/* + * 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.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; + +/** + * Serializer based on standard Java serialization + */ +public class JavaSerializer implements Serializer { + @Override public ByteBuffer serialize(Object obj) { + if (obj == null) + return null; + + ByteArrayOutputStream stream = null; + ObjectOutputStream out = null; + + try { + stream = new ByteArrayOutputStream(4096); + out = new ObjectOutputStream(stream); + + out.writeObject(obj); + out.flush(); + + return ByteBuffer.wrap(stream.toByteArray()); + } + catch (IOException e) { + throw new RuntimeException("Failed to serialize object of the class '" + obj.getClass().getName() + "'", e); + } + finally { + if (out != null) { + try { + out.close(); + } + catch (Throwable ignored) { + } + } + + if (stream != null) { + try { + stream.close(); + } + catch (Throwable ignored) { + } + } + } + } + + @Override public Object deserialize(ByteBuffer buffer) { + ByteArrayInputStream stream = null; + ObjectInputStream in = null; + + try { + stream = new ByteArrayInputStream(buffer.array()); + in = new ObjectInputStream(stream); + return in.readObject(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to deserialize object from byte stream", e); + } + finally { + if (in != null) { + try { + in.close(); + } + catch (Throwable ignored) { + } + } + + if (stream != null) { + try { + stream.close(); + } + catch (Throwable ignored) { + } + } + } + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/KryoSerializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/KryoSerializer.java new file mode 100644 index 0000000..adaa1b8 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/KryoSerializer.java @@ -0,0 +1,105 @@ +/* + * 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.serializer; + +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.nio.ByteBuffer; + +/** + * Serializer based on Kryo serialization + */ +public class KryoSerializer implements Serializer { + private ThreadLocal kryos = new ThreadLocal() { + protected Kryo initialValue() { + return new Kryo(); + }; + }; + + @Override public ByteBuffer serialize(Object obj) { + if (obj == null) + return null; + + ByteArrayOutputStream stream = null; + Output out = null; + + try { + stream = new ByteArrayOutputStream(4096); + out = new Output(stream); + + kryos.get().writeClassAndObject(out, obj); + out.flush(); + + return ByteBuffer.wrap(stream.toByteArray()); + } + catch (Throwable e) { + throw new RuntimeException("Failed to serialize object of the class '" + obj.getClass().getName() + "'", e); + } + finally { + if (out != null) { + try { + out.close(); + } + catch (Throwable ignored) { + } + } + + if (stream != null) { + try { + stream.close(); + } + catch (Throwable ignored) { + } + } + } + } + + @Override public Object deserialize(ByteBuffer buffer) { + ByteArrayInputStream stream = null; + Input in = null; + + try { + stream = new ByteArrayInputStream(buffer.array()); + in = new Input(stream); + return kryos.get().readClassAndObject(in); + } + catch (Throwable e) { + throw new RuntimeException("Failed to deserialize object from byte stream", e); + } + finally { + if (in != null) { + try { + in.close(); + } + catch (Throwable ignored) { + } + } + + if (stream != null) { + try { + stream.close(); + } + catch (Throwable ignored) { + } + } + } + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/Serializer.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/Serializer.java new file mode 100644 index 0000000..da58434 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/serializer/Serializer.java @@ -0,0 +1,31 @@ +/* + * 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.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 { + public ByteBuffer serialize(Object obj); + + public Object deserialize(ByteBuffer buffer); +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/BatchExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/BatchExecutionAssistant.java new file mode 100644 index 0000000..a0392d4 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/BatchExecutionAssistant.java @@ -0,0 +1,49 @@ +/* + * 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.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.utils.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 { + public boolean tableExistenceRequired(); + + public String getStatement(); + + public BoundStatement bindStatement(PreparedStatement statement, V obj); + + public KeyValuePersistenceSettings getPersistenceSettings(); + + public String operationName(); + + public void process(Row row, int sequenceNumber); + + public boolean alreadyProcessed(int sequenceNumber); + + public int processedCount(); + + public R processedData(); +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSession.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSession.java new file mode 100644 index 0000000..ca36b38 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSession.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.cache.store.cassandra.utils.session; + +import java.io.Closeable; + +/** + * Wrapper around Cassandra driver session, to automatically handle + * - keyspace and table absence exceptions + * - timeout exceptions + * - batch operations + */ +public interface CassandraSession extends Closeable { + public V execute(ExecutionAssistant assistant); + + public R execute(BatchExecutionAssistant controller, Iterable data); +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSessionImpl.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSessionImpl.java new file mode 100644 index 0000000..0971966 --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/CassandraSessionImpl.java @@ -0,0 +1,516 @@ +/* + * 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.session; + +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 java.io.IOException; +import java.util.LinkedList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import javax.cache.Cache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cache.store.cassandra.utils.common.CassandraHelper; +import org.apache.ignite.cache.store.cassandra.utils.persistence.KeyValuePersistenceSettings; +import org.apache.ignite.cache.store.cassandra.utils.session.pool.SessionPool; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; + +/** + * Implementation for ${@link org.apache.ignite.cache.store.cassandra.utils.session.CassandraSession} + */ +public class CassandraSessionImpl implements CassandraSession { + private static final int CQL_EXECUTION_ATTEMPTS_COUNT = 20; + private static final int CQL_EXECUTION_ATTEMPTS_TIMEOUT = 2000; + + private volatile Cluster.Builder builder; + private volatile Session session; + private volatile int refCount = 0; + + private Integer fetchSize; + private ConsistencyLevel readConsistency; + private ConsistencyLevel writeConsistency; + private IgniteLogger logger; + + private final AtomicInteger handlersCount = new AtomicInteger(-1); + + public CassandraSessionImpl(Cluster.Builder builder, Integer fetchSize, ConsistencyLevel readConsistency, + ConsistencyLevel writeConsistency, IgniteLogger logger) { + this.builder = builder; + this.fetchSize = fetchSize; + this.readConsistency = readConsistency; + this.writeConsistency = writeConsistency; + this.logger = logger; + } + + @Override public V execute(ExecutionAssistant assistant) { + int attempt = 0; + Throwable error = null; + String errorMsg = "Failed to execute Cassandra CQL statement: " + assistant.getStatement(); + + incrementSessionRefs(); + + try { + while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + error = null; + + try { + PreparedStatement preparedSt = prepareStatement(assistant.getStatement(), + assistant.getPersistenceSettings(), assistant.tableExistenceRequired()); + + if (preparedSt == null) + return null; + + Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt)); + ResultSet result = session().execute(statement); + + Row row = result == null || !result.iterator().hasNext() ? null : result.iterator().next(); + + return row == null ? null : assistant.process(row); + } + catch (Throwable e) { + if (CassandraHelper.isTableAbsenceError(e)) { + if (!assistant.tableExistenceRequired()) { + logger.warning(errorMsg, e); + return null; + } + + handleTableAbsenceError(assistant.getPersistenceSettings()); + } + else if (CassandraHelper.isHostsAvailabilityError(e)) + handleHostsAvailabilityError(e, attempt, errorMsg); + else if (!CassandraHelper.isPreparedStatementClusterError(e)) + throw new IgniteException(errorMsg, e); + + if (logger != null && !CassandraHelper.isPreparedStatementClusterError(e)) + logger.warning(errorMsg, e); + + error = e; + } + + attempt++; + } + } + finally { + decrementSessionRefs(); + } + + throw new IgniteException("Failed to execute Cassandra CQL statement: " + assistant.getStatement(), error); + } + + @Override public R execute(BatchExecutionAssistant assistant, Iterable 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); + + int dataSize = -1; + + incrementSessionRefs(); + + try { + while (dataSize != assistant.processedCount() && error != null && attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + boolean tableAbsenceErrorFlag = false; + boolean hostsAvailabilityErrorFlag = false; + boolean prepStatementErrorFlag = false; + error = null; + + List> futureResults = new LinkedList<>(); + + PreparedStatement preparedSt = prepareStatement(assistant.getStatement(), + assistant.getPersistenceSettings(), assistant.tableExistenceRequired()); + + if (preparedSt == null) + return null; + + int sequenceNumber = 0; + + for (V obj : data) { + if (assistant.alreadyProcessed(sequenceNumber)) + continue; + + Statement statement = tuneStatementExecutionOptions(assistant.bindStatement(preparedSt, obj)); + ResultSetFuture future = session().executeAsync(statement); + futureResults.add(new CacheEntryImpl<>(sequenceNumber, future)); + + sequenceNumber++; + } + + dataSize = sequenceNumber; + + for (Cache.Entry futureResult : futureResults) { + try { + ResultSet resultSet = futureResult.getValue().getUninterruptibly(); + Row row = resultSet != null && resultSet.iterator().hasNext() ? resultSet.iterator().next() : null; + + if (row != null) + assistant.process(row, futureResult.getKey()); + } + catch (Throwable e) { + if (CassandraHelper.isTableAbsenceError(e)) + tableAbsenceErrorFlag = true; + else if (CassandraHelper.isHostsAvailabilityError(e)) + hostsAvailabilityErrorFlag = true; + else if (CassandraHelper.isPreparedStatementClusterError(e)) + prepStatementErrorFlag = true; + + error = error == null || !CassandraHelper.isPreparedStatementClusterError(e) ? e : error; + } + } + + // if no errors occurred it means that operation successfully completed and we can return + if (error == null) + return assistant.processedData(); + + // if there were no errors which we know how to handle, we will not try next attempts and terminate + if (!tableAbsenceErrorFlag && !hostsAvailabilityErrorFlag && !prepStatementErrorFlag) + throw new IgniteException(errorMsg, error); + + if (logger != null && !CassandraHelper.isPreparedStatementClusterError(error)) + logger.warning(errorMsg, error); + + // if there are only table absence errors and it is not required for the operation we can return + if (tableAbsenceErrorFlag && !assistant.tableExistenceRequired()) + return assistant.processedData(); + + if (tableAbsenceErrorFlag) + handleTableAbsenceError(assistant.getPersistenceSettings()); + + if (hostsAvailabilityErrorFlag) + handleHostsAvailabilityError(error, attempt, errorMsg); + + attempt++; + } + } + finally { + decrementSessionRefs(); + } + + errorMsg = "Failed to process " + (dataSize - assistant.processedCount()) + + " of " + dataSize + " elements during " + assistant.operationName() + + " operation with Cassandra"; + + if (assistant.processedCount() == 0) + throw new IgniteException(errorMsg, error); + + if (logger != null) + logger.warning(errorMsg, error); + + return assistant.processedData(); + } + + @Override public synchronized void close() throws IOException { + if (decrementSessionRefs() == 0 && session != null) { + SessionPool.put(this, session); + session = null; + } + } + + private synchronized void refresh() { + //make sure that session removed from the pool + SessionPool.get(this); + + //closing and reopening session + CassandraHelper.closeSession(session); + session = null; + session(); + } + + private synchronized Session session() { + if (session != null) + return session; + + session = SessionPool.get(this); + + if (session != null) + return session; + + try { + return session = builder.build().connect(); + } + catch (Throwable e) { + throw new IgniteException("Failed to establish session with Cassandra database", e); + } + } + + private synchronized void incrementSessionRefs() { + refCount++; + } + + private synchronized int decrementSessionRefs() { + if (refCount != 0) + refCount--; + + return refCount; + } + + private PreparedStatement prepareStatement(String statement, KeyValuePersistenceSettings settings, + boolean tableExistenceRequired) { + + int attempt = 0; + Throwable error = null; + String errorMsg = "Failed to prepare Cassandra CQL statement: " + statement; + + incrementSessionRefs(); + + try { + while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + try { + return session().prepare(statement); + } + catch (Throwable e) { + if (CassandraHelper.isTableAbsenceError(e)) { + if (!tableExistenceRequired) + return null; + + handleTableAbsenceError(settings); + } + else if (CassandraHelper.isHostsAvailabilityError(e)) + handleHostsAvailabilityError(e, attempt, errorMsg); + else + throw new IgniteException(errorMsg, e); + + error = e; + } + + attempt++; + } + } + finally { + decrementSessionRefs(); + } + + throw new IgniteException(errorMsg, error); + } + + 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) { + try { + logger.info("Creating Cassandra keyspace '" + settings.getKeyspace() + "'"); + session().execute(settings.getKeyspaceDDLStatement()); + logger.info("Cassandra keyspace '" + settings.getKeyspace() + "' was successfully created"); + return; + } + catch (AlreadyExistsException ignored) { + logger.info("Cassandra keyspace '" + settings.getKeyspace() + "' already exist"); + return; + } + catch (Throwable e) { + if (!CassandraHelper.isHostsAvailabilityError(e)) + throw new IgniteException(errorMsg, e); + + handleHostsAvailabilityError(e, attempt, errorMsg); + + error = e; + } + + attempt++; + } + + throw new IgniteException(errorMsg, error); + } + + private void createTable(KeyValuePersistenceSettings settings) { + int attempt = 0; + Throwable error = null; + String errorMsg = "Failed to create Cassandra table '" + settings.getTableFullName() + "'"; + + while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + try { + logger.info("Creating Cassandra table '" + settings.getTableFullName() + "'"); + session().execute(settings.getTableDDLStatement()); + logger.info("Cassandra table '" + settings.getTableFullName() + "' was successfully created"); + return; + } + catch (AlreadyExistsException ignored) { + logger.info("Cassandra table '" + settings.getTableFullName() + "' already exist"); + return; + } + catch (Throwable e) { + if (!CassandraHelper.isHostsAvailabilityError(e) && !CassandraHelper.isKeyspaceAbsenceError(e)) + throw new IgniteException(errorMsg, e); + + if (CassandraHelper.isKeyspaceAbsenceError(e)) { + logger.warning("Failed to create Cassandra table '" + settings.getTableFullName() + + "' cause appropriate keyspace doesn't exist", e); + createKeyspace(settings); + } + else if (CassandraHelper.isHostsAvailabilityError(e)) { + handleHostsAvailabilityError(e, attempt, errorMsg); + } + + error = e; + } + + attempt++; + } + + throw new IgniteException(errorMsg, error); + } + + private void createTableIndexes(KeyValuePersistenceSettings settings) { + if (settings.getIndexDDLStatements() == null || settings.getIndexDDLStatements().isEmpty()) + return; + + int attempt = 0; + Throwable error = null; + String errorMsg = "Failed to create indexes for Cassandra table " + settings.getTableFullName(); + + while (attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + try { + logger.info("Creating indexes for Cassandra table '" + settings.getTableFullName() + "'"); + + for (String statement : settings.getIndexDDLStatements()) { + try { + session().execute(statement); + } + catch (AlreadyExistsException ignored) { + } + catch (Throwable e) { + if (!(e instanceof InvalidQueryException) || !e.getMessage().equals("Index already exists")) + throw new IgniteException(errorMsg, e); + } + } + + logger.info("Indexes for Cassandra table '" + settings.getTableFullName() + "' were successfully created"); + + return; + } + catch (Throwable e) { + if (CassandraHelper.isHostsAvailabilityError(e)) + handleHostsAvailabilityError(e, attempt, errorMsg); + else if (CassandraHelper.isTableAbsenceError(e)) + createTable(settings); + else + throw new IgniteException(errorMsg, e); + + error = e; + } + + attempt++; + } + + throw new IgniteException(errorMsg, error); + } + + private Statement tuneStatementExecutionOptions(Statement statement) { + String query = ""; + + if (statement instanceof BoundStatement) + query = ((BoundStatement)statement).preparedStatement().getQueryString().trim().toLowerCase(); + else if (statement instanceof PreparedStatement) + query = ((PreparedStatement)statement).getQueryString().trim().toLowerCase(); + + boolean readStatement = query.startsWith("select"); + boolean writeStatement = statement instanceof Batch || statement instanceof BatchStatement || + query.startsWith("insert") || query.startsWith("delete") || query.startsWith("update"); + + if (readStatement && readConsistency != null) + statement.setConsistencyLevel(readConsistency); + + if (writeStatement && writeConsistency != null) + statement.setConsistencyLevel(writeConsistency); + + if (fetchSize != null) + statement.setFetchSize(fetchSize); + + return statement; + } + + private void handleTableAbsenceError(KeyValuePersistenceSettings settings) { + int handlerNumber = handlersCount.incrementAndGet(); + + try { + synchronized (handlersCount) { + // Oooops... I am not the first thread who tried to handle table absence problem + if (handlerNumber != 0) + return; + + RuntimeException error = new IgniteException("Failed to create Cassandra table " + settings.getTableFullName()); + + int attempt = 0; + + while (error != null && attempt < CQL_EXECUTION_ATTEMPTS_COUNT) { + error = null; + + try { + createKeyspace(settings); + createTable(settings); + createTableIndexes(settings); + } + catch (Throwable e) { + if (CassandraHelper.isHostsAvailabilityError(e)) + handleHostsAvailabilityError(e, attempt, null); + else + throw new IgniteException("Failed to create Cassandra table " + settings.getTableFullName(), e); + + error = e instanceof RuntimeException ? (RuntimeException)e : new IgniteException(e); + } + + attempt++; + } + + if (error != null) + throw error; + } + } + finally { + if (handlerNumber == 0) + handlersCount.set(-1); + } + } + + private boolean handleHostsAvailabilityError(Throwable e, int attempt, String msg) { + if (attempt >= CQL_EXECUTION_ATTEMPTS_COUNT) + throw msg == null ? new IgniteException(e) : new IgniteException(msg, e); + + if (attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 4 || + attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 3 || + attempt == CQL_EXECUTION_ATTEMPTS_COUNT / 2 || + attempt == CQL_EXECUTION_ATTEMPTS_COUNT - 1) { + refresh(); + return true; + } + + try { + Thread.sleep(CQL_EXECUTION_ATTEMPTS_TIMEOUT); + } + catch (InterruptedException ex) { + throw new IgniteException(msg, e); + } + + return false; + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/ExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/ExecutionAssistant.java new file mode 100644 index 0000000..319afbb --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/ExecutionAssistant.java @@ -0,0 +1,42 @@ +/* + * 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.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.utils.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 { + public boolean tableExistenceRequired(); + + public String getStatement(); + + public BoundStatement bindStatement(PreparedStatement statement); + + public KeyValuePersistenceSettings getPersistenceSettings(); + + public String operationName(); + + public R process(Row row); +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/GenericBatchExecutionAssistant.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/GenericBatchExecutionAssistant.java new file mode 100644 index 0000000..5d8556d --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/GenericBatchExecutionAssistant.java @@ -0,0 +1,59 @@ +/* + * 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.session; + +import com.datastax.driver.core.Row; +import java.util.HashSet; +import java.util.Set; + +/** + * Implementation of the ${@link org.apache.ignite.cache.store.cassandra.utils.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 { + private Set processed = new HashSet<>(); + + @Override public void process(Row row, int sequenceNumber) { + if (processed.contains(sequenceNumber)) + return; + + processed.add(sequenceNumber); + + process(row); + } + + @Override public boolean alreadyProcessed(int sequenceNumber) { + return processed.contains(sequenceNumber); + } + + @Override public int processedCount() { + return processed.size(); + } + + @Override public R processedData() { + return null; + } + + @Override public boolean tableExistenceRequired() { + return false; + } + + protected void process(Row row) { + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionPool.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionPool.java new file mode 100644 index 0000000..eeedecf --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionPool.java @@ -0,0 +1,151 @@ +/* + * 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.session.pool; + +import com.datastax.driver.core.Session; +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 org.apache.ignite.cache.store.cassandra.utils.session.CassandraSessionImpl; + +/** + * Cassandra driver sessions pool + */ +public class SessionPool { + private static class SessionMonitor extends Thread { + @Override public void run() { + try { + while (true) { + try { + Thread.sleep(SLEEP_TIMEOUT); + } + catch (InterruptedException ignored) { + return; + } + + List> expiredSessions = new LinkedList<>(); + + int sessionsCount; + + synchronized (sessions) { + sessionsCount = 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 (sessionsCount == expiredSessions.size()) + return; + } + } + finally { + release(); + } + } + } + + private static final long SLEEP_TIMEOUT = 60000; // 1 minute + + private static final Map sessions = new HashMap<>(); + + private static SessionMonitor monitorSingleton; + + static { + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override public void run() { + release(); + } + }); + } + + public static void put(CassandraSessionImpl cassandraSession, Session driverSession) { + if (cassandraSession == null || driverSession == null) + return; + + SessionWrapper old; + + synchronized (sessions) { + old = sessions.put(cassandraSession, new SessionWrapper(driverSession)); + + 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(); + } + + public static Session get(CassandraSessionImpl cassandraSession) { + if (cassandraSession == null) + return null; + + SessionWrapper wrapper; + + synchronized (sessions) { + wrapper = sessions.remove(cassandraSession); + } + + return wrapper == null ? null : wrapper.driverSession(); + } + + public static void release() { + Collection wrappers; + + synchronized (sessions) { + try { + if (sessions.size() == 0) + return; + + wrappers = new LinkedList<>(); + + for (SessionWrapper wrapper : sessions.values()) + wrappers.add(wrapper); + + sessions.clear(); + } + finally { + if (!(Thread.currentThread() instanceof SessionMonitor) && monitorSingleton != null) { + try { + monitorSingleton.interrupt(); + } + catch (Throwable ignored) { + } + } + } + } + + for (SessionWrapper wrapper : wrappers) + wrapper.release(); + } +} diff --git a/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionWrapper.java b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionWrapper.java new file mode 100644 index 0000000..09b022b --- /dev/null +++ b/modules/cassandra/src/main/java/org/apache/ignite/cache/store/cassandra/utils/session/pool/SessionWrapper.java @@ -0,0 +1,49 @@ +/* + * 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.session.pool; + +import com.datastax.driver.core.Session; +import org.apache.ignite.cache.store.cassandra.utils.common.CassandraHelper; + +/** + * Wrapper for Cassandra driver session, responsible for monitoring session expiration and its closing + */ +public class SessionWrapper { + public static final long EXPIRATION_TIMEOUT = 300000; //5 minutes + + private Session session; + private long time; + + public SessionWrapper(Session session) { + this.session = session; + this.time = System.currentTimeMillis(); + } + + public boolean expired() { + return System.currentTimeMillis() - time > EXPIRATION_TIMEOUT; + } + + public Session driverSession() { + return session; + } + + public void release() { + CassandraHelper.closeSession(session); + session = null; + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java new file mode 100644 index 0000000..a78b690 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceLoadTest.java @@ -0,0 +1,76 @@ +/* + * 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.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 = Logger.getLogger("CassandraLoadTests"); + + public static void main(String[] args) { + try { + LOGGER.info("Cassandra load tests execution started"); + + LoadTestDriver driver = new CassandraDirectPersistenceLoadTest(); + + CassandraHelper.dropTestKeyspaces(); + + driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME); + + driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME); + + driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME); + + CassandraHelper.dropTestKeyspaces(); + + driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME); + + CassandraHelper.dropTestKeyspaces(); + + LOGGER.info("Cassandra load tests execution completed"); + } + finally { + CassandraHelper.releaseCassandraResources(); + } + } + + @Override protected Logger logger() { + return LOGGER; + } + + @Override protected Object setup(String loggerName) { + return CacheStoreHelper.createCacheStore( + TestsHelper.getLoadTestsCacheName(), + TestsHelper.getLoadTestsPersistenceSettings(), + CassandraHelper.getAdminDataSource(), + Logger.getLogger(loggerName)); + } + +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java new file mode 100644 index 0000000..7f06ac5 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/CassandraDirectPersistenceTest.java @@ -0,0 +1,321 @@ +/* + * 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.Map; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.pojos.Person; +import org.apache.ignite.tests.pojos.PersonId; +import org.apache.ignite.tests.utils.CacheStoreHelper; +import org.apache.ignite.tests.utils.CassandraHelper; +import org.apache.ignite.tests.utils.TestsHelper; +import org.apache.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 = Logger.getLogger(CassandraDirectPersistenceTest.class.getName()); + + @BeforeClass + public static void setUpClass() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + return; + } + + 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() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + return; + } + + try { + CassandraHelper.dropTestKeyspaces(); + } + finally { + CassandraHelper.releaseCassandraResources(); + } + } + + @Test + @SuppressWarnings("unchecked") + public void primitiveStrategyTest() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping primitiveStrategyTest test"); + return; + } + + CacheStore store1 = CacheStoreHelper.createCacheStore("intTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml"), + CassandraHelper.getAdminDataSource()); + + CacheStore store2 = CacheStoreHelper.createCacheStore("stringTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml"), + CassandraHelper.getAdminDataSource()); + + Collection> intEntries = TestsHelper.generateIntegersEntries(); + Collection> strEntries = TestsHelper.generateStringsEntries(); + + LOGGER.info("Running PRIMITIVE strategy write tests"); + + LOGGER.info("Running single operation write tests"); + store1.write(intEntries.iterator().next()); + store2.write(strEntries.iterator().next()); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + store1.writeAll(intEntries); + store2.writeAll(strEntries); + LOGGER.info("Bulk operation write tests passed"); + + LOGGER.info("PRIMITIVE strategy write tests passed"); + + LOGGER.info("Running PRIMITIVE strategy read tests"); + + LOGGER.info("Running single operation read tests"); + + Integer intValue = (Integer)store1.load(intEntries.iterator().next().getKey()); + if (!intEntries.iterator().next().getValue().equals(intValue)) + throw new RuntimeException("Integer values was incorrectly deserialized from Cassandra"); + + String strValue = (String)store2.load(strEntries.iterator().next().getKey()); + if (!strEntries.iterator().next().getValue().equals(strValue)) + throw new RuntimeException("String values was incorrectly deserialized from Cassandra"); + + LOGGER.info("Single operation read tests passed"); + + LOGGER.info("Running bulk operation read tests"); + + Map intValues = store1.loadAll(TestsHelper.getKeys(intEntries)); + if (!TestsHelper.checkCollectionsEqual(intValues, intEntries)) + throw new RuntimeException("Integer values was incorrectly deserialized from Cassandra"); + + Map strValues = store2.loadAll(TestsHelper.getKeys(strEntries)); + if (!TestsHelper.checkCollectionsEqual(strValues, strEntries)) + throw new RuntimeException("String values 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"); + + store1.delete(intEntries.iterator().next().getKey()); + store1.deleteAll(TestsHelper.getKeys(intEntries)); + + store2.delete(strEntries.iterator().next().getKey()); + store2.deleteAll(TestsHelper.getKeys(strEntries)); + + LOGGER.info("PRIMITIVE strategy delete tests passed"); + } + + @Test + @SuppressWarnings("unchecked") + public void blobStrategyTest() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping blobStrategyTest test"); + return; + } + + CacheStore store1 = CacheStoreHelper.createCacheStore("intTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml"), + CassandraHelper.getAdminDataSource()); + + CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml"), + CassandraHelper.getAdminDataSource()); + + CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml"), + CassandraHelper.getAdminDataSource()); + + Collection> intEntries = TestsHelper.generateIntegersEntries(); + Collection> personEntries = TestsHelper.generateIntegersPersonsEntries(); + + LOGGER.info("Running BLOB strategy write tests"); + + LOGGER.info("Running single operation write tests"); + store1.write(intEntries.iterator().next()); + store2.write(personEntries.iterator().next()); + store3.write(personEntries.iterator().next()); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + store1.writeAll(intEntries); + store2.writeAll(personEntries); + store3.writeAll(personEntries); + LOGGER.info("Bulk operation write tests passed"); + + LOGGER.info("BLOB strategy write tests passed"); + + LOGGER.info("Running BLOB strategy read tests"); + + LOGGER.info("Running single operation read tests"); + + Integer intValue = (Integer)store1.load(intEntries.iterator().next().getKey()); + if (!intEntries.iterator().next().getValue().equals(intValue)) + throw new RuntimeException("Integer values was incorrectly deserialized from Cassandra"); + + Person personValue = (Person)store2.load(personEntries.iterator().next().getKey()); + if (!personEntries.iterator().next().getValue().equals(personValue)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + personValue = (Person)store3.load(personEntries.iterator().next().getKey()); + if (!personEntries.iterator().next().getValue().equals(personValue)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + LOGGER.info("Single operation read tests passed"); + + LOGGER.info("Running bulk operation read tests"); + + Map intValues = store1.loadAll(TestsHelper.getKeys(intEntries)); + if (!TestsHelper.checkCollectionsEqual(intValues, intEntries)) + throw new RuntimeException("Integer values was incorrectly deserialized from Cassandra"); + + Map personValues = store2.loadAll(TestsHelper.getKeys(personEntries)); + if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + personValues = store3.loadAll(TestsHelper.getKeys(personEntries)); + if (!TestsHelper.checkPersonCollectionsEqual(personValues, personEntries, false)) + throw new RuntimeException("Person values 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"); + + store1.delete(intEntries.iterator().next().getKey()); + store1.deleteAll(TestsHelper.getKeys(intEntries)); + + 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() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping pojoStrategyTest test"); + return; + } + + CacheStore store1 = CacheStoreHelper.createCacheStore("intTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml"), + CassandraHelper.getAdminDataSource()); + + CacheStore store2 = CacheStoreHelper.createCacheStore("personTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml"), + CassandraHelper.getAdminDataSource()); + + CacheStore store3 = CacheStoreHelper.createCacheStore("personTypes", + new ClassPathResource("org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml"), + CassandraHelper.getAdminDataSource()); + + Collection> entries1 = TestsHelper.generateIntegersPersonsEntries(); + Collection> entries2 = TestsHelper.generatePersonIdsPersonsEntries(); + Collection> entries3 = TestsHelper.generatePersonIdsPersonsEntries(); + + LOGGER.info("Running POJO strategy write tests"); + + LOGGER.info("Running single operation write tests"); + store1.write(entries1.iterator().next()); + store2.write(entries2.iterator().next()); + store3.write(entries3.iterator().next()); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + store1.writeAll(entries1); + store2.writeAll(entries2); + store3.writeAll(entries3); + LOGGER.info("Bulk operation write tests passed"); + + LOGGER.info("POJO strategy write tests passed"); + + LOGGER.info("Running POJO strategy read tests"); + + LOGGER.info("Running single operation read tests"); + + Person person = (Person)store1.load(entries1.iterator().next().getKey()); + if (!entries1.iterator().next().getValue().equalsPrimitiveFields(person)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + person = (Person)store2.load(entries2.iterator().next().getKey()); + if (!entries2.iterator().next().getValue().equalsPrimitiveFields(person)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + person = (Person)store3.load(entries3.iterator().next().getKey()); + if (!entries3.iterator().next().getValue().equals(person)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + LOGGER.info("Single operation read tests passed"); + + LOGGER.info("Running bulk operation read tests"); + + Map persons = store1.loadAll(TestsHelper.getKeys(entries1)); + if (!TestsHelper.checkPersonCollectionsEqual(persons, entries1, true)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + persons = store2.loadAll(TestsHelper.getKeys(entries2)); + if (!TestsHelper.checkPersonCollectionsEqual(persons, entries2, true)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + persons = store3.loadAll(TestsHelper.getKeys(entries3)); + if (!TestsHelper.checkPersonCollectionsEqual(persons, entries3, false)) + throw new RuntimeException("Person values was incorrectly deserialized from Cassandra"); + + LOGGER.info("Bulk operation read 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)); + + LOGGER.info("POJO strategy delete tests passed"); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.java new file mode 100644 index 0000000..cd5032b --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreLoadTest.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.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.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 = Logger.getLogger("IgniteLoadTests"); + + public static void main(String[] args) { + try { + LOGGER.info("Ignite load tests execution started"); + + LoadTestDriver driver = new IgnitePersistentStoreLoadTest(); + + CassandraHelper.dropTestKeyspaces(); + + driver.runTest("WRITE", WriteWorker.class, WriteWorker.LOGGER_NAME); + + driver.runTest("READ", ReadWorker.class, ReadWorker.LOGGER_NAME); + + driver.runTest("BULK_READ", BulkReadWorker.class, BulkReadWorker.LOGGER_NAME); + + CassandraHelper.dropTestKeyspaces(); + + driver.runTest("BULK_WRITE", BulkWriteWorker.class, BulkWriteWorker.LOGGER_NAME); + + CassandraHelper.dropTestKeyspaces(); + + LOGGER.info("Ignite load tests execution completed"); + } + finally { + CassandraHelper.releaseCassandraResources(); + } + } + + @Override protected Logger logger() { + return LOGGER; + } + + @Override protected Object setup(String loggerName) { + return Ignition.start(TestsHelper.getLoadTestsIgniteConfig()); + } + + @Override protected void tearDown(Object obj) { + Ignite ignite = (Ignite)obj; + + if (ignite != null) + ignite.close(); + } + +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java new file mode 100644 index 0000000..5c09eaa --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/IgnitePersistentStoreTest.java @@ -0,0 +1,317 @@ +/* + * 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.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.tests.pojos.Person; +import org.apache.ignite.tests.pojos.PersonId; +import org.apache.ignite.tests.utils.CassandraHelper; +import org.apache.ignite.tests.utils.TestsHelper; +import org.apache.log4j.Logger; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +/** + * 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 = Logger.getLogger(IgnitePersistentStoreTest.class.getName()); + + @BeforeClass + public static void setUpClass() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + return; + } + + 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() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + return; + } + + try { + CassandraHelper.dropTestKeyspaces(); + } + finally { + CassandraHelper.releaseCassandraResources(); + } + } + + @Test + public void primitiveStrategyTest() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping primitiveStrategyTest test"); + return; + } + + Ignition.stopAll(true); + + Map intMap = TestsHelper.generateIntegersMap(); + 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 intCache = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache strCache = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + + LOGGER.info("Running single operation write tests"); + intCache.put(1, 1); + strCache.put("1", "1"); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + intCache.putAll(intMap); + 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 intCache = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache strCache = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + + LOGGER.info("Running single operation read tests"); + + Integer intVal = intCache.get(1); + if (!intVal.equals(intMap.get(1))) + throw new RuntimeException("Integer 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 intMap1 = intCache.getAll(intMap.keySet()); + if (!TestsHelper.checkMapsEqual(intMap, intMap1)) + throw new RuntimeException("Integer 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"); + + intCache.remove(1); + intCache.removeAll(intMap.keySet()); + + strCache.remove("1"); + strCache.removeAll(strMap.keySet()); + + LOGGER.info("PRIMITIVE strategy delete tests passed"); + } + } + + @Test + public void blobStrategyTest() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping blobStrategyTest test"); + return; + } + + Ignition.stopAll(true); + + Map intMap = TestsHelper.generateIntegersMap(); + Map personMap = TestsHelper.generateIntegersPersonsMap(); + + LOGGER.info("Running BLOB strategy write tests"); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/blob/ignite-config.xml")) { + IgniteCache intCache = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache personCache = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + + LOGGER.info("Running single operation write tests"); + intCache.put(1, 1); + personCache.put(1, TestsHelper.generateRandomPerson()); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + intCache.putAll(intMap); + 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 intCache = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache personCache = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + + LOGGER.info("Running single operation read tests"); + + Integer intVal = intCache.get(1); + if (!intVal.equals(intMap.get(1))) + throw new RuntimeException("Integer value was incorrectly deserialized from Cassandra"); + + Person person = personCache.get(1); + if (!person.equals(personMap.get(1))) + 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 intMap1 = intCache.getAll(intMap.keySet()); + if (!TestsHelper.checkMapsEqual(intMap, intMap1)) + throw new RuntimeException("Integer 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"); + + intCache.remove(1); + intCache.removeAll(intMap.keySet()); + + personCache.remove(1); + personCache.removeAll(personMap.keySet()); + + LOGGER.info("BLOB strategy delete tests passed"); + } + } + + @Test + public void pojoStrategyTest() { + if (CassandraHelper.getAdminPassword().isEmpty() || CassandraHelper.getRegularPassword().isEmpty()) { + LOGGER.info("Cassandra passwords weren't specified thus skipping pojoStrategyTest test"); + return; + } + + Ignition.stopAll(true); + + LOGGER.info("Running POJO strategy write tests"); + + Map personMap1 = TestsHelper.generateIntegersPersonsMap(); + Map personMap2 = TestsHelper.generatePersonIdsPersonsMap(); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) { + IgniteCache personCache1 = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache personCache2 = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + IgniteCache personCache3 = ignite.getOrCreateCache(new CacheConfiguration("cache3")); + + LOGGER.info("Running single operation write tests"); + personCache1.put(1, TestsHelper.generateRandomPerson()); + personCache2.put(TestsHelper.generateRandomPersonId(), TestsHelper.generateRandomPerson()); + personCache3.put(TestsHelper.generateRandomPersonId(), TestsHelper.generateRandomPerson()); + LOGGER.info("Single operation write tests passed"); + + LOGGER.info("Running bulk operation write tests"); + personCache1.putAll(personMap1); + personCache2.putAll(personMap2); + personCache3.putAll(personMap2); + LOGGER.info("Bulk operation write tests passed"); + } + + LOGGER.info("POJO strategy write tests passed"); + + Ignition.stopAll(true); + + try (Ignite ignite = Ignition.start("org/apache/ignite/tests/persistence/pojo/ignite-config.xml")) { + LOGGER.info("Running POJO strategy read tests"); + + IgniteCache personCache1 = ignite.getOrCreateCache(new CacheConfiguration("cache1")); + IgniteCache personCache2 = ignite.getOrCreateCache(new CacheConfiguration("cache2")); + IgniteCache personCache3 = ignite.getOrCreateCache(new CacheConfiguration("cache3")); + + LOGGER.info("Running single operation read tests"); + Person person = personCache1.get(1); + if (!person.equalsPrimitiveFields(personMap1.get(1))) + throw new RuntimeException("Person value was incorrectly deserialized from Cassandra"); + + PersonId id = personMap2.keySet().iterator().next(); + + person = personCache2.get(id); + if (!person.equalsPrimitiveFields(personMap2.get(id))) + throw new RuntimeException("Person value was incorrectly deserialized from Cassandra"); + + person = personCache3.get(id); + if (!person.equals(personMap2.get(id))) + 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 persons1 = personCache1.getAll(personMap1.keySet()); + if (!TestsHelper.checkPersonMapsEqual(persons1, personMap1, true)) + throw new RuntimeException("Integer values batch was incorrectly deserialized from Cassandra"); + + Map persons2 = personCache2.getAll(personMap2.keySet()); + if (!TestsHelper.checkPersonMapsEqual(persons2, personMap2, true)) + throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra"); + + Map persons3 = personCache3.getAll(personMap2.keySet()); + if (!TestsHelper.checkPersonMapsEqual(persons3, personMap2, false)) + throw new RuntimeException("Person values batch was incorrectly deserialized from Cassandra"); + + LOGGER.info("Bulk operation read tests passed"); + + LOGGER.info("POJO strategy read tests passed"); + + LOGGER.info("Running POJO strategy delete tests"); + + personCache1.remove(1); + personCache1.removeAll(personMap1.keySet()); + + personCache2.remove(id); + personCache2.removeAll(personMap2.keySet()); + + personCache3.remove(id); + personCache3.removeAll(personMap2.keySet()); + + LOGGER.info("POJO strategy delete tests passed"); + } + } + +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java new file mode 100644 index 0000000..db49741 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Generator.java @@ -0,0 +1,26 @@ +/* + * 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.load; + +/** + * Generator abstraction which could be used by tests to generate next key/value pair for Ignite cache + * from provided int number (which sequentially incremented in load test driver loop) + */ +public interface Generator { + public Object generate(int i); +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java new file mode 100644 index 0000000..1e52a6b --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/IntGenerator.java @@ -0,0 +1,28 @@ +/* + * 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.load; + +/** + * Implementation of ${@link org.apache.ignite.tests.load.Generator} generating + * ${@link Integer} instance + */ +public class IntGenerator implements Generator { + @Override public Object generate(int i) { + return i; + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java new file mode 100644 index 0000000..0ecb36d --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/LoadTestDriver.java @@ -0,0 +1,146 @@ +/* + * 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.load; + +import java.lang.reflect.Constructor; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.cassandra.utils.common.SystemHelper; +import org.apache.ignite.tests.utils.TestsHelper; +import org.apache.log4j.Logger; + +/** + * Basic load test driver to be inherited by specific implementation for particular use-case + */ +public abstract class LoadTestDriver { + public void runTest(String testName, Class clazz, String loggerName) { + logger().info("Running " + testName + " test"); + + Object config = setup(loggerName); + + try { + + List workers = new LinkedList<>(); + int startPosition = 0; + + logger().info("Starting workers"); + + for (int i = 0; i < TestsHelper.getLoadTestsThreadsCount(); i++) { + Worker worker = createWorker(clazz, config, startPosition, startPosition + 10000000); + workers.add(worker); + worker.setName(testName + "-worker-" + i); + worker.start(); + startPosition += 10000001; + } + + logger().info("Workers started"); + logger().info("Waiting for workers to complete"); + + List failedWorkers = new LinkedList<>(); + + for (Worker worker : workers) { + boolean failed = false; + + try { + worker.join(); + } + catch (Throwable e) { + logger().error("Worker " + worker.getName() + " waiting interrupted", e); + failed = true; + } + + if (failed || worker.isFailed()) + failedWorkers.add(worker.getName()); + } + + printTestResultsHeader(testName, failedWorkers); + printTestResultsStatistics(testName, workers); + } + finally { + tearDown(config); + } + } + + protected abstract Logger logger(); + + protected abstract Object setup(String loggerName); + + protected void tearDown(Object obj) { + } + + @SuppressWarnings("unchecked") + private Worker createWorker(Class clazz, Object config, int startPosition, int endPosition) { + try { + Class configClass = config instanceof Ignite ? Ignite.class : CacheStore.class; + Constructor constr = clazz.getConstructor(configClass, int.class, int.class); + return (Worker)constr.newInstance(config, startPosition, endPosition); + } + catch (Throwable e) { + logger().error("Failed to instantiate worker of class '" + clazz.getName() + "'", e); + throw new RuntimeException("Failed to instantiate worker of class '" + clazz.getName() + "'", e); + } + } + + private void printTestResultsHeader(String testName, List failedWorkers) { + if (failedWorkers.isEmpty()) { + logger().info(testName + " test execution successfully completed."); + return; + } + + if (failedWorkers.size() == TestsHelper.getLoadTestsThreadsCount()) { + logger().error(testName + " test execution totally failed."); + return; + } + + String strFailedWorkers = ""; + + for (String workerName : failedWorkers) { + if (!strFailedWorkers.isEmpty()) + strFailedWorkers += ", "; + + strFailedWorkers += workerName; + } + + logger().warn(testName + " test execution completed, but " + failedWorkers.size() + " of " + + TestsHelper.getLoadTestsThreadsCount() + " workers failed. Failed workers: " + strFailedWorkers); + } + + @SuppressWarnings("StringBufferReplaceableByString") + private void printTestResultsStatistics(String testName, List workers) { + int count = 0; + int speed = 0; + + for (Worker worker : workers) { + count += worker.getMsgCountTotal(); + speed += worker.getSpeed(); + } + + StringBuilder builder = new StringBuilder(); + builder.append(SystemHelper.LINE_SEPARATOR); + builder.append("-------------------------------------------------"); + builder.append(SystemHelper.LINE_SEPARATOR); + builder.append(testName).append(" test statistics").append(SystemHelper.LINE_SEPARATOR); + builder.append("Messages: ").append(count).append(SystemHelper.LINE_SEPARATOR); + builder.append("Speed: ").append(speed).append(" msg/sec").append(SystemHelper.LINE_SEPARATOR); + builder.append("-------------------------------------------------"); + + logger().info(builder.toString()); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java new file mode 100644 index 0000000..fa440c2 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonGenerator.java @@ -0,0 +1,41 @@ +/* + * 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.load; + +import java.util.Date; +import java.util.LinkedList; +import java.util.List; +import org.apache.ignite.tests.pojos.*; + +/** + * Implementation of ${@link org.apache.ignite.tests.load.Generator} generating + * ${@link org.apache.ignite.tests.pojos.Person} instance + */ +public class PersonGenerator implements Generator { + private static final Date DATE = new Date(); + + private static final List PHONES = new LinkedList(){{ + add("1234567"); + add("7654321"); + add("1289054"); + }}; + + @Override public Object generate(int i) { + return new Person(Integer.toString(i), Integer.toString(i), i, i % 2 == 0, i, i, DATE, PHONES); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java new file mode 100644 index 0000000..07e3caa --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/PersonIdGenerator.java @@ -0,0 +1,30 @@ +/* + * 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.load; + +import org.apache.ignite.tests.pojos.PersonId; + +/** + * Implementation of ${@link org.apache.ignite.tests.load.Generator} generating + * ${@link org.apache.ignite.tests.pojos.PersonId} instance + */ +public class PersonIdGenerator implements Generator { + @Override public Object generate(int i) { + return new PersonId(Integer.toString(i), Integer.toString(i), i); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java new file mode 100644 index 0000000..2e4d9a8 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/StringGenerator.java @@ -0,0 +1,27 @@ +/* + * 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.load; + +/** + * Implementation of ${@link org.apache.ignite.tests.load.Generator} generating ${@link String} instance + */ +public class StringGenerator implements Generator { + @Override public Object generate(int i) { + return Integer.toString(i); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java new file mode 100644 index 0000000..751545a --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/Worker.java @@ -0,0 +1,307 @@ +/* + * 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.load; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.Ignition; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.cassandra.utils.common.SystemHelper; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.utils.TestsHelper; +import org.apache.log4j.Logger; + +/** + * Worker thread abstraction to be inherited by specific load test implementation + */ +public abstract class Worker extends Thread { + private static final SimpleDateFormat TIME_FORMATTER = new SimpleDateFormat("hh:mm:ss"); + + private long testStartTime; + + boolean warmup = TestsHelper.getLoadTestsWarmupPeriod() != 0; + + private volatile long warmupStartTime = 0; + private volatile long warmupFinishTime = 0; + + private volatile long startTime = 0; + private volatile long finishTime = 0; + + private volatile int warmupMsgProcessed = 0; + private volatile int warmupSleepCount = 0; + + private volatile int msgProcessed = 0; + private volatile int sleepCount = 0; + + private Throwable executionError; + + private long statReportedTime; + + private CacheStore cacheStore; + private Ignite ignite; + private IgniteCache igniteCache; + + private Logger logger; + private int startPosition; + private int endPosition; + + public Worker(CacheStore cacheStore, int startPosition, int endPosition) { + this.cacheStore = cacheStore; + this.logger = Logger.getLogger(loggerName()); + this.startPosition = startPosition; + this.endPosition = endPosition; + } + + public Worker(Ignite ignite, int startPosition, int endPosition) { + this.ignite = ignite; + this.logger = Logger.getLogger(loggerName()); + this.startPosition = startPosition; + this.endPosition = endPosition; + } + + @SuppressWarnings("unchecked") + @Override public void run() { + try { + if (cacheStore != null) + execute(); + else { + igniteCache = ignite.getOrCreateCache(new CacheConfiguration(TestsHelper.getLoadTestsCacheName())); + execute(); + } + } + catch (Throwable e) { + executionError = e; + throw new RuntimeException("Test execution abnormally terminated", e); + } + } + + public boolean isFailed() { + return executionError != null; + } + + public int getSpeed() { + if (msgProcessed == 0) + return 0; + + long finish = finishTime != 0 ? finishTime : System.currentTimeMillis(); + long duration = (finish - startTime - sleepCount * TestsHelper.getLoadTestsRequestsLatency()) / 1000; + + return duration == 0 ? msgProcessed : msgProcessed / (int)duration; + } + + public int getMsgCountTotal() { + return warmupMsgProcessed + msgProcessed; + } + + protected abstract String loggerName(); + + protected abstract boolean batchMode(); + + protected void process(CacheStore cacheStore, CacheEntryImpl entry) { + throw new UnsupportedOperationException("Single message processing is not supported"); + } + + protected void process(IgniteCache cache, Object key, Object value) { + throw new UnsupportedOperationException("Single message processing is not supported"); + } + + protected void process(CacheStore cacheStore, Collection entries) { + throw new UnsupportedOperationException("Batch processing is not supported"); + } + + protected void process(IgniteCache cache, Map map) { + throw new UnsupportedOperationException("Batch processing is not supported"); + } + + @SuppressWarnings("unchecked") + private void execute() throws InterruptedException { + testStartTime = System.currentTimeMillis(); + + logger.info("Test execution started"); + + if (warmup) + logger.info("Warm up period started"); + + warmupStartTime = warmup ? testStartTime : 0; + startTime = !warmup ? testStartTime : 0; + + statReportedTime = testStartTime; + + int counter = startPosition; + Object key = TestsHelper.generateLoadTestsKey(counter); + Object val = TestsHelper.generateLoadTestsValue(counter); + List batchList = new ArrayList<>(TestsHelper.getBulkOperationSize()); + Map batchMap = new HashMap(TestsHelper.getBulkOperationSize()); + + try { + while (true) { + if (System.currentTimeMillis() - testStartTime > TestsHelper.getLoadTestsExecutionTime()) { + break; + } + + if (warmup && System.currentTimeMillis() - testStartTime > TestsHelper.getLoadTestsWarmupPeriod()) { + warmupFinishTime = System.currentTimeMillis(); + startTime = warmupFinishTime; + warmup = false; + logger.info("Warm up period completed"); + } + + if (!batchMode()) { + if (cacheStore != null) + doWork(new CacheEntryImpl(key, val)); + else + doWork(key, val); + } + else if (batchList.size() == TestsHelper.getBulkOperationSize() || + batchMap.size() == TestsHelper.getBulkOperationSize()) { + if (cacheStore != null) + doWork(batchList); + else + doWork(batchMap); + + batchMap.clear(); + batchList.clear(); + } + + if (counter == endPosition) + counter = startPosition; + else + counter++; + + key = TestsHelper.generateLoadTestsKey(counter); + val = TestsHelper.generateLoadTestsValue(counter); + + if (batchMode()) { + if (cacheStore != null) + batchList.add(new CacheEntryImpl(key, val)); + else + batchMap.put(key, val); + } + + reportStatistics(); + } + } + finally { + warmupFinishTime = warmupFinishTime != 0 ? warmupFinishTime : System.currentTimeMillis(); + finishTime = System.currentTimeMillis(); + reportTestCompletion(); + } + } + + private void doWork(CacheEntryImpl entry) throws InterruptedException { + process(cacheStore, entry); + updateMetrics(1); + } + + private void doWork(Object key, Object value) throws InterruptedException { + process(igniteCache, key, value); + updateMetrics(1); + } + + private void doWork(Collection entries) throws InterruptedException { + process(cacheStore, entries); + updateMetrics(entries.size()); + } + + private void doWork(Map entries) throws InterruptedException { + process(igniteCache, entries); + updateMetrics(entries.size()); + } + + private int getWarmUpSpeed() { + if (warmupMsgProcessed == 0) + return 0; + + long finish = warmupFinishTime != 0 ? warmupFinishTime : System.currentTimeMillis(); + long duration = (finish - warmupStartTime - warmupSleepCount * TestsHelper.getLoadTestsRequestsLatency()) / 1000; + + return duration == 0 ? warmupMsgProcessed : warmupMsgProcessed / (int)duration; + } + + private void updateMetrics(int itemsProcessed) throws InterruptedException { + if (warmup) + warmupMsgProcessed += itemsProcessed; + else + msgProcessed += itemsProcessed; + + if (TestsHelper.getLoadTestsRequestsLatency() > 0) + Thread.sleep(TestsHelper.getLoadTestsRequestsLatency()); + + if (warmup) + warmupSleepCount++; + else + sleepCount++; + } + + private void reportStatistics() { + // statistics should be reported only every 30 seconds + if (System.currentTimeMillis() - statReportedTime < 30000) + return; + + statReportedTime = System.currentTimeMillis(); + + int completed = (int)(statReportedTime - testStartTime) * 100 / TestsHelper.getLoadTestsExecutionTime(); + + if (warmup) { + logger.info("Warm up messages processed " + warmupMsgProcessed + ", " + + "speed " + getWarmUpSpeed() + " msg/sec, " + completed + "% completed"); + } + else { + logger.info("Messages processed " + msgProcessed + ", " + + "speed " + getSpeed() + " msg/sec, " + completed + "% completed"); + } + } + + private void reportTestCompletion() { + StringBuilder builder = new StringBuilder(); + + if (executionError != null) + builder.append("Test execution abnormally terminated. "); + else + builder.append("Test execution successfully completed. "); + + builder.append("Statistics: ").append(SystemHelper.LINE_SEPARATOR); + builder.append("Start time: ").append(TIME_FORMATTER.format(testStartTime)).append(SystemHelper.LINE_SEPARATOR); + builder.append("Finish time: ").append(TIME_FORMATTER.format(finishTime)).append(SystemHelper.LINE_SEPARATOR); + builder.append("Duration: ").append((finishTime - testStartTime) / 1000).append(" sec") + .append(SystemHelper.LINE_SEPARATOR); + + if (TestsHelper.getLoadTestsWarmupPeriod() > 0) { + builder.append("Warm up period: ").append(TestsHelper.getLoadTestsWarmupPeriod() / 1000) + .append(" sec").append(SystemHelper.LINE_SEPARATOR); + builder.append("Warm up processed messages: ").append(warmupMsgProcessed).append(SystemHelper.LINE_SEPARATOR); + builder.append("Warm up processing speed: ").append(getWarmUpSpeed()) + .append(" msg/sec").append(SystemHelper.LINE_SEPARATOR); + } + + builder.append("Processed messages: ").append(msgProcessed).append(SystemHelper.LINE_SEPARATOR); + builder.append("Processing speed: ").append(getSpeed()).append(" msg/sec"); + + if (executionError != null) + logger.error(builder.toString(), executionError); + else + logger.info(builder.toString()); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.java new file mode 100644 index 0000000..472098e --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkReadWorker.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.tests.load.cassandra; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.load.Worker; +import org.apache.ignite.tests.utils.TestsHelper; + +/** + * Cassandra direct load tests worker for bulk read operation CacheStore.load + */ +public class BulkReadWorker extends Worker { + public static final String LOGGER_NAME = "CassandraBulkReadLoadTest"; + + private List keys = new ArrayList<>(TestsHelper.getBulkOperationSize()); + + public BulkReadWorker(CacheStore cacheStore, int startPosition, int endPosition) { + super(cacheStore, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return true; + } + + @SuppressWarnings("unchecked") + @Override protected void process(CacheStore cacheStore, Collection entries) { + keys.clear(); + + for (CacheEntryImpl entry : entries) + keys.add(entry.getKey()); + + cacheStore.loadAll(keys); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.java new file mode 100644 index 0000000..bfe9246 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/BulkWriteWorker.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.tests.load.cassandra; + +import java.util.Collection; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.load.Worker; + +/** + * Cassandra direct load tests worker for bulk write operation CacheStore.writeAll + */ +public class BulkWriteWorker extends Worker { + public static final String LOGGER_NAME = "CassandraBulkWriteLoadTest"; + + public BulkWriteWorker(CacheStore cacheStore, int startPosition, int endPosition) { + super(cacheStore, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return true; + } + + @SuppressWarnings("unchecked") + @Override protected void process(CacheStore cacheStore, Collection entries) { + cacheStore.writeAll(entries); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java new file mode 100644 index 0000000..5021e37 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/ReadWorker.java @@ -0,0 +1,46 @@ +/* + * 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.load.cassandra; + +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.load.Worker; + +/** + * Cassandra direct load tests worker for read operation CacheStore.load + */ +public class ReadWorker extends Worker { + public static final String LOGGER_NAME = "CassandraReadLoadTest"; + + public ReadWorker(CacheStore cacheStore, int startPosition, int endPosition) { + super(cacheStore, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return false; + } + + @SuppressWarnings("unchecked") + @Override protected void process(CacheStore cacheStore, CacheEntryImpl entry) { + cacheStore.load(entry.getKey()); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java new file mode 100644 index 0000000..dd87410 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/cassandra/WriteWorker.java @@ -0,0 +1,46 @@ +/* + * 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.load.cassandra; + +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.load.Worker; + +/** + * Cassandra direct load tests worker for write operation CacheStore.write + */ +public class WriteWorker extends Worker { + public static final String LOGGER_NAME = "CassandraWriteLoadTest"; + + public WriteWorker(CacheStore cacheStore, int startPosition, int endPosition) { + super(cacheStore, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return false; + } + + @SuppressWarnings("unchecked") + @Override protected void process(CacheStore cacheStore, CacheEntryImpl entry) { + cacheStore.write(entry); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.java new file mode 100644 index 0000000..108c8fc --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkReadWorker.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.tests.load.ignite; + +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.tests.load.Worker; + +/** + * Ignite load tests worker for bulk read operation CacheStore.loadAll + */ +public class BulkReadWorker extends Worker { + public static final String LOGGER_NAME = "IgniteBulkReadLoadTest"; + + public BulkReadWorker(Ignite ignite, int startPosition, int endPosition) { + super(ignite, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return true; + } + + @SuppressWarnings("unchecked") + @Override protected void process(IgniteCache cache, Map entries) { + cache.getAll(entries.keySet()); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.java new file mode 100644 index 0000000..dd934f0 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/BulkWriteWorker.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.tests.load.ignite; + +import java.util.Map; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.tests.load.Worker; + +/** + * Ignite load tests worker for bulk read operation CacheStore.writeAll + */ +public class BulkWriteWorker extends Worker { + public static final String LOGGER_NAME = "IgniteBulkWriteLoadTest"; + + public BulkWriteWorker(Ignite ignite, int startPosition, int endPosition) { + super(ignite, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return true; + } + + @SuppressWarnings("unchecked") + @Override protected void process(IgniteCache cache, Map entries) { + cache.putAll(entries); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java new file mode 100644 index 0000000..5591ab7 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/ReadWorker.java @@ -0,0 +1,46 @@ +/* + * 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.load.ignite; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.tests.load.Worker; + +/** + * Ignite load tests worker for read operation CacheStore.load + */ +public class ReadWorker extends Worker { + public static final String LOGGER_NAME = "IgniteReadLoadTest"; + + public ReadWorker(Ignite ignite, int startPosition, int endPosition) { + super(ignite, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return false; + } + + @SuppressWarnings("unchecked") + @Override protected void process(IgniteCache cache, Object key, Object val) { + cache.get(key); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java new file mode 100644 index 0000000..3c40efc --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/load/ignite/WriteWorker.java @@ -0,0 +1,46 @@ +/* + * 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.load.ignite; + +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.tests.load.Worker; + +/** + * Ignite load tests worker for write operation CacheStore.write + */ +public class WriteWorker extends Worker { + public static final String LOGGER_NAME = "IgniteWriteLoadTest"; + + public WriteWorker(Ignite ignite, int startPosition, int endPosition) { + super(ignite, startPosition, endPosition); + } + + @Override protected String loggerName() { + return LOGGER_NAME; + } + + @Override protected boolean batchMode() { + return false; + } + + @SuppressWarnings("unchecked") + @Override protected void process(IgniteCache cache, Object key, Object val) { + cache.put(key, val); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java new file mode 100644 index 0000000..3abc38e --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/Person.java @@ -0,0 +1,215 @@ +/* + * 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.pojos; + +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.util.Date; +import java.util.List; + +/** + * Simple POJO which could be stored as a value in Ignite cache + */ +public class Person implements Externalizable { + private String firstName; + private String lastName; + private int age; + private boolean married; + private long height; + private float weight; + private Date birthDate; + private List phones; + + @SuppressWarnings("UnusedDeclaration") + public Person() { + } + + public Person(String firstName, String lastName, int age, boolean married, + long height, float weight, Date birthDate, List phones) { + this.firstName = firstName; + this.lastName = lastName; + this.age = age; + this.married = married; + this.height = height; + this.weight = weight; + this.birthDate = birthDate; + this.phones = phones; + } + + @Override public void writeExternal(ObjectOutput out) throws IOException { + out.writeObject(firstName); + out.writeObject(lastName); + out.writeInt(age); + out.writeBoolean(married); + out.writeLong(height); + out.writeFloat(weight); + out.writeObject(birthDate); + out.writeObject(phones); + } + + @SuppressWarnings("unchecked") + @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + firstName = (String)in.readObject(); + lastName = (String)in.readObject(); + age = in.readInt(); + married = in.readBoolean(); + height = in.readLong(); + weight = in.readFloat(); + birthDate = (Date)in.readObject(); + phones = (List)in.readObject(); + } + + @SuppressWarnings("SimplifiableIfStatement") + @Override public boolean equals(Object obj) { + if (obj == null || !(obj instanceof Person)) + return false; + + Person person = (Person)obj; + + if ((firstName != null && !firstName.equals(person.firstName)) || + (person.firstName != null && !person.firstName.equals(firstName))) { + return false; + } + + if ((lastName != null && !lastName.equals(person.lastName)) || + (person.lastName != null && !person.lastName.equals(lastName))) { + return false; + } + + if ((birthDate != null && !birthDate.equals(person.birthDate)) || + (person.birthDate != null && !person.birthDate.equals(birthDate))) { + return false; + } + + if ((phones != null && !phones.equals(person.phones)) || + (person.phones != null && !person.phones.equals(phones))) { + return false; + } + + return age == person.age && married == person.married && + height == person.height && weight == person.weight; + } + + @SuppressWarnings("SimplifiableIfStatement") + public boolean equalsPrimitiveFields(Object obj) { + if (obj == null || !(obj instanceof Person)) + return false; + + Person person = (Person)obj; + + if ((firstName != null && !firstName.equals(person.firstName)) || + (person.firstName != null && !person.firstName.equals(firstName))) { + return false; + } + + if ((lastName != null && !lastName.equals(person.lastName)) || + (person.lastName != null && !person.lastName.equals(lastName))) { + return false; + } + + if ((birthDate != null && !birthDate.equals(person.birthDate)) || + (person.birthDate != null && !person.birthDate.equals(birthDate))) { + return false; + } + + return age == person.age && married == person.married && + height == person.height && weight == person.weight; + } + + @SuppressWarnings("UnusedDeclaration") + public void setFirstName(String name) { + firstName = name; + } + + @SuppressWarnings("UnusedDeclaration") + public String getFirstName() { + return firstName; + } + + @SuppressWarnings("UnusedDeclaration") + public void setLastName(String name) { + lastName = name; + } + + @SuppressWarnings("UnusedDeclaration") + public String getLastName() { + return lastName; + } + + @SuppressWarnings("UnusedDeclaration") + public void setAge(int age) { + this.age = age; + } + + @SuppressWarnings("UnusedDeclaration") + public int getAge() { + return age; + } + + @SuppressWarnings("UnusedDeclaration") + public void setMarried(boolean married) { + this.married = married; + } + + @SuppressWarnings("UnusedDeclaration") + public boolean getMarried() { + return married; + } + + @SuppressWarnings("UnusedDeclaration") + public void setHeight(long height) { + this.height = height; + } + + @SuppressWarnings("UnusedDeclaration") + public long getHeight() { + return height; + } + + @SuppressWarnings("UnusedDeclaration") + public void setWeight(float weight) { + this.weight = weight; + } + + @SuppressWarnings("UnusedDeclaration") + public float getWeight() { + return weight; + } + + @SuppressWarnings("UnusedDeclaration") + public void setBirthDate(Date date) { + birthDate = date; + } + + @SuppressWarnings("UnusedDeclaration") + public Date getBirthDate() { + return birthDate; + } + + @SuppressWarnings("UnusedDeclaration") + public void setPhones(List phones) { + this.phones = phones; + } + + @SuppressWarnings("UnusedDeclaration") + public List getPhones() { + return phones; + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.java new file mode 100644 index 0000000..38e6e14 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/pojos/PersonId.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.tests.pojos; + +import java.io.Serializable; + +/** + * Simple POJO which could be stored as a key in Ignite cache + */ +public class PersonId implements Serializable { + private String companyCode; + private String departmentCode; + private int personNumber; + + @SuppressWarnings("UnusedDeclaration") + public PersonId() { + } + + public PersonId(String companyCode, String departmentCode, int personNumber) { + this.companyCode = companyCode; + this.departmentCode = departmentCode; + this.personNumber = personNumber; + } + + @SuppressWarnings("SimplifiableIfStatement") + @Override public boolean equals(Object obj) { + if (obj == null || !(obj instanceof PersonId)) + return false; + + PersonId id = (PersonId)obj; + + if ((companyCode != null && !companyCode.equals(id.companyCode)) || + (id.companyCode != null && !id.companyCode.equals(companyCode))) { + return false; + } + + if ((companyCode != null && !companyCode.equals(id.companyCode)) || + (id.companyCode != null && !id.companyCode.equals(companyCode))) { + return false; + } + + return personNumber == id.personNumber; + } + + @Override public int hashCode() { + String code = (companyCode == null ? "" : companyCode) + + (departmentCode == null ? "" : departmentCode) + + personNumber; + + return code.hashCode(); + } + + @SuppressWarnings("UnusedDeclaration") + public void setCompanyCode(String code) { + companyCode = code; + } + + @SuppressWarnings("UnusedDeclaration") + public String getCompanyCode() { + return companyCode; + } + + @SuppressWarnings("UnusedDeclaration") + public void setDepartmentCode(String code) { + departmentCode = code; + } + + @SuppressWarnings("UnusedDeclaration") + public String getDepartmentCode() { + return departmentCode; + } + + @SuppressWarnings("UnusedDeclaration") + public void setPersonNumber(int number) { + personNumber = number; + } + + @SuppressWarnings("UnusedDeclaration") + public int getPersonNumber() { + return personNumber; + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.java new file mode 100644 index 0000000..0d6474e --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CacheStoreHelper.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.utils; + +import java.lang.reflect.Field; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.cassandra.CassandraCacheStore; +import org.apache.ignite.cache.store.cassandra.utils.datasource.DataSource; +import org.apache.ignite.cache.store.cassandra.utils.persistence.KeyValuePersistenceSettings; +import org.apache.ignite.logger.log4j.Log4JLogger; +import org.apache.log4j.Logger; +import org.springframework.core.io.Resource; + +/** + * Helper class utilized by unit tests to get appropriate instance of ${@link CacheStore} + */ +public class CacheStoreHelper { + private static final Logger LOGGER = Logger.getLogger(CacheStoreHelper.class.getName()); + + public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, DataSource connection) { + return createCacheStore(cacheName, persistenceSettings, connection, LOGGER); + } + + public static CacheStore createCacheStore(String cacheName, Resource persistenceSettings, + DataSource connection, Logger logger) { + CassandraCacheStore cacheStore = + new CassandraCacheStore<>(connection, new KeyValuePersistenceSettings(persistenceSettings)); + + try { + Field sessionField = CassandraCacheStore.class.getDeclaredField("storeSession"); + Field loggerField = CassandraCacheStore.class.getDeclaredField("logger"); + + sessionField.setAccessible(true); + loggerField.setAccessible(true); + + sessionField.set(cacheStore, new TestCacheSession(cacheName)); + loggerField.set(cacheStore, new Log4JLogger(logger)); + } + catch (Throwable e) { + throw new RuntimeException("Failed to initialize test Ignite cache store", e); + } + + return cacheStore; + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java new file mode 100644 index 0000000..dc1a170 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraAdminCredentials.java @@ -0,0 +1,34 @@ +/* + * 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.utils; + +import org.apache.ignite.cache.store.cassandra.utils.datasource.Credentials; + +/** + * Implementation of ${@link org.apache.ignite.cache.store.cassandra.utils.datasource.Credentials} providing + * admin user/password to establish Cassandra session + */ +public class CassandraAdminCredentials implements Credentials { + @Override public String getUser() { + return CassandraHelper.getAdminUser(); + } + + @Override public String getPassword() { + return CassandraHelper.getAdminPassword(); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java new file mode 100644 index 0000000..b163687 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraHelper.java @@ -0,0 +1,281 @@ +/* + * 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.utils; + +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import java.net.InetAddress; +import java.net.InetSocketAddress; +import java.util.LinkedList; +import java.util.List; +import java.util.ResourceBundle; +import org.apache.ignite.cache.store.cassandra.utils.datasource.DataSource; +import org.apache.ignite.cache.store.cassandra.utils.session.pool.SessionPool; +import org.springframework.context.ApplicationContext; +import org.springframework.context.support.ClassPathXmlApplicationContext; + +/** + * Helper class providing bunch of utility methods to work with Cassandra + */ +public class CassandraHelper { + private static final ResourceBundle CREDENTIALS = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/credentials"); + private static final ResourceBundle CONNECTION = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/connection"); + private static final ResourceBundle KEYSPACES = ResourceBundle.getBundle("org/apache/ignite/tests/cassandra/keyspaces"); + + private static final ApplicationContext connectionContext = new ClassPathXmlApplicationContext("org/apache/ignite/tests/cassandra/connection-settings.xml"); + + private static DataSource adminDataSource; + private static DataSource regularDataSource; + + private static Cluster adminCluster; + private static Cluster regularCluster; + private static Session adminSession; + private static Session regularSession; + + public static String getAdminUser() { + return CREDENTIALS.getString("admin.user"); + } + + public static String getAdminPassword() { + return CREDENTIALS.getString("admin.password"); + } + + public static String getRegularUser() { + return CREDENTIALS.getString("regular.user"); + } + + public static String getRegularPassword() { + return CREDENTIALS.getString("regular.password"); + } + + public static String[] getTestKeyspaces() { + return KEYSPACES.getString("keyspaces").split(","); + } + + public static String[] getContactPointsArray() { + String[] points = CONNECTION.getString("contact.points").split(","); + + if (points.length == 0) + throw new RuntimeException("No Cassandra contact points specified"); + + for (int i = 0; i < points.length; i++) + points[i] = points[i].trim(); + + return points; + } + + public static List getContactPoints() { + String[] points = getContactPointsArray(); + + List contactPoints = new LinkedList<>(); + + for (String point : points) { + if (point.contains(":")) + continue; + + try { + contactPoints.add(InetAddress.getByName(point)); + } + catch (Throwable e) { + throw new IllegalArgumentException("Incorrect contact point '" + point + + "' specified for Cassandra cache storage", e); + } + } + + return contactPoints; + } + + public static List getContactPointsWithPorts() { + String[] points = getContactPointsArray(); + + List contactPoints = new LinkedList<>(); + + for (String point : points) { + if (!point.contains(":")) + continue; + + String[] chunks = point.split(":"); + + try { + contactPoints.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); + } + } + + return contactPoints; + } + + public static void dropTestKeyspaces() { + String[] keyspaces = getTestKeyspaces(); + + for (String keyspace : keyspaces) { + try { + executeWithAdminCredentials("DROP KEYSPACE IF EXISTS " + keyspace + ";"); + } + catch (Throwable e) { + throw new RuntimeException("Failed to drop keyspace: " + keyspace, e); + } + } + } + + public static ResultSet executeWithAdminCredentials(String statement, Object... args) { + if (args == null || args.length == 0) + return adminSession().execute(statement); + + PreparedStatement ps = adminSession().prepare(statement); + return adminSession().execute(ps.bind(args)); + } + + @SuppressWarnings("UnusedDeclaration") + public static ResultSet executeWithRegularCredentials(String statement, Object... args) { + if (args == null || args.length == 0) + return regularSession().execute(statement); + + PreparedStatement ps = regularSession().prepare(statement); + return regularSession().execute(ps.bind(args)); + } + + @SuppressWarnings("UnusedDeclaration") + public static ResultSet executeWithAdminCredentials(Statement statement) { + return adminSession().execute(statement); + } + + @SuppressWarnings("UnusedDeclaration") + public static ResultSet executeWithRegularCredentials(Statement statement) { + return regularSession().execute(statement); + } + + public static synchronized DataSource getAdminDataSource() { + if (adminDataSource != null) + return adminDataSource; + + return adminDataSource = (DataSource)connectionContext.getBean("cassandraAdminDataSource"); + } + + @SuppressWarnings("UnusedDeclaration") + public static synchronized DataSource getRegularDataSource() { + if (regularDataSource != null) + return regularDataSource; + + return regularDataSource = (DataSource)connectionContext.getBean("cassandraRegularDataSource"); + } + + public static void testAdminConnection() { + try { + adminSession(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to check admin connection to Cassandra", e); + } + } + + public static void testRegularConnection() { + try { + regularSession(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to check regular connection to Cassandra", e); + } + } + + public static synchronized void releaseCassandraResources() { + try { + if (adminSession != null && !adminSession.isClosed()) + adminSession.close(); + } + catch (Throwable ignored) { + } + finally { + adminSession = null; + } + + try { + if (adminCluster != null && !adminCluster.isClosed()) + adminCluster.close(); + } + catch (Throwable ignored) { + } + finally { + adminCluster = null; + } + + try { + if (regularSession != null && !regularSession.isClosed()) + regularSession.close(); + } + catch (Throwable ignored) { + } + finally { + regularSession = null; + } + + try { + if (regularCluster != null && !regularCluster.isClosed()) + regularCluster.close(); + } + catch (Throwable ignored) { + } + finally { + regularCluster = null; + } + + SessionPool.release(); + } + + private static synchronized Session adminSession() { + if (adminSession != null) + return adminSession; + + try { + Cluster.Builder builder = Cluster.builder(); + builder = builder.withCredentials(getAdminUser(), getAdminPassword()); + builder.addContactPoints(getContactPoints()); + builder.addContactPointsWithPorts(getContactPointsWithPorts()); + + adminCluster = builder.build(); + return adminSession = adminCluster.connect(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to create admin session to Cassandra database", e); + } + } + + private static synchronized Session regularSession() { + if (regularSession != null) + return regularSession; + + try { + Cluster.Builder builder = Cluster.builder(); + builder = builder.withCredentials(getRegularUser(), getRegularPassword()); + builder.addContactPoints(getContactPoints()); + builder.addContactPointsWithPorts(getContactPointsWithPorts()); + + regularCluster = builder.build(); + return regularSession = regularCluster.connect(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to create regular session to Cassandra database", e); + } + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java new file mode 100644 index 0000000..a38c91f --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/CassandraRegularCredentials.java @@ -0,0 +1,34 @@ +/* + * 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.utils; + +import org.apache.ignite.cache.store.cassandra.utils.datasource.Credentials; + +/** + * Implementation of ${@link org.apache.ignite.cache.store.cassandra.utils.datasource.Credentials} providing + * regular user/password to establish Cassandra session + */ +public class CassandraRegularCredentials implements Credentials { + @Override public String getUser() { + return CassandraHelper.getRegularUser(); + } + + @Override public String getPassword() { + return CassandraHelper.getRegularPassword(); + } +} diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java new file mode 100644 index 0000000..af89510 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestCacheSession.java @@ -0,0 +1,77 @@ +/* + * 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.utils; + +import java.util.Map; +import org.apache.ignite.cache.store.CacheStoreSession; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.transactions.Transaction; +import org.jetbrains.annotations.Nullable; + +/** + * Test implementation of ${@link CacheStoreSession} for the unit tests purposes + */ +public class TestCacheSession implements CacheStoreSession { + + private String cacheName; + private Transaction tx; + private Map props; + private Object attachment; + + public TestCacheSession(String cacheName) { + this.cacheName = cacheName; + } + + @SuppressWarnings("UnusedDeclaration") + public void newSession(@Nullable Transaction tx) { + this.tx = tx; + props = null; + } + + @Nullable @Override public Transaction transaction() { + return tx; + } + + @Override public boolean isWithinTransaction() { + return transaction() != null; + } + + @SuppressWarnings("unchecked") + @Override public Object attach(@Nullable Object attachment) { + Object prev = this.attachment; + this.attachment = attachment; + return prev; + } + + @SuppressWarnings("unchecked") + @Nullable @Override public T attachment() { + return (T)attachment; + } + + @SuppressWarnings("unchecked") + @Override public Map properties() { + if (props == null) + props = U.newHashMap(1); + + return (Map)props; + } + + @Nullable @Override public String cacheName() { + return cacheName; + } +} \ No newline at end of file diff --git a/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java new file mode 100644 index 0000000..f9f5437 --- /dev/null +++ b/modules/cassandra/src/test/java/org/apache/ignite/tests/utils/TestsHelper.java @@ -0,0 +1,312 @@ +/* + * 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.utils; + +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.ResourceBundle; +import org.apache.ignite.internal.processors.cache.CacheEntryImpl; +import org.apache.ignite.tests.load.Generator; +import org.apache.ignite.tests.pojos.Person; +import org.apache.ignite.tests.pojos.PersonId; +import org.springframework.core.io.ClassPathResource; + +/** + * Helper class for all tests + */ +public class TestsHelper { + private static final String LETTERS_ALPHABET = "ABCDEFGHIJKLMNOPQRSTUVWXYZ"; + private static final String NUMBERS_ALPHABET = "0123456789"; + private static final Random RANDOM = new Random(System.currentTimeMillis()); + + private static final ResourceBundle TESTS_SETTINGS = ResourceBundle.getBundle("tests"); + + private static final int BULK_OPERATION_SIZE = Integer.parseInt(TESTS_SETTINGS.getString("bulk.operation.size")); + private static final String LOAD_TESTS_CACHE_NAME = TESTS_SETTINGS.getString("load.tests.cache.name"); + private static final int LOAD_TESTS_THREADS_COUNT = Integer.parseInt(TESTS_SETTINGS.getString("load.tests.threads.count")); + private static final int LOAD_TESTS_WARMUP_PERIOD = Integer.parseInt(TESTS_SETTINGS.getString("load.tests.warmup.period")); + private static final int LOAD_TESTS_EXECUTION_TIME = Integer.parseInt(TESTS_SETTINGS.getString("load.tests.execution.time")); + private static final int LOAD_TESTS_REQUESTS_LATENCY = Integer.parseInt(TESTS_SETTINGS.getString("load.tests.requests.latency")); + private static final String LOAD_TESTS_PERSISTENCE_SETTINGS = TESTS_SETTINGS.getString("load.tests.persistence.settings"); + private static final String LOAD_TESTS_IGNITE_CONFIG = TESTS_SETTINGS.getString("load.tests.ignite.config"); + private static final Generator LOAD_TESTS_KEY_GENERATOR; + private static final Generator LOAD_TESTS_VALUE_GENERATOR; + + static { + try { + LOAD_TESTS_KEY_GENERATOR = (Generator)Class.forName(TESTS_SETTINGS.getString("load.tests.key.generator")).newInstance(); + LOAD_TESTS_VALUE_GENERATOR = (Generator)Class.forName(TESTS_SETTINGS.getString("load.tests.value.generator")).newInstance(); + } + catch (Throwable e) { + throw new RuntimeException("Failed to initialize TestsHelper", e); + } + } + + public static int getLoadTestsThreadsCount() { + return LOAD_TESTS_THREADS_COUNT; + } + + public static int getLoadTestsWarmupPeriod() { + return LOAD_TESTS_WARMUP_PERIOD; + } + + public static int getLoadTestsExecutionTime() { + return LOAD_TESTS_EXECUTION_TIME; + } + + public static int getLoadTestsRequestsLatency() { + return LOAD_TESTS_REQUESTS_LATENCY; + } + + public static ClassPathResource getLoadTestsPersistenceSettings() { + return new ClassPathResource(LOAD_TESTS_PERSISTENCE_SETTINGS); + } + + public static String getLoadTestsIgniteConfig() { + return LOAD_TESTS_IGNITE_CONFIG; + } + + public static int getBulkOperationSize() { + return BULK_OPERATION_SIZE; + } + + public static String getLoadTestsCacheName() { + return LOAD_TESTS_CACHE_NAME; + } + + public static Object generateLoadTestsKey(int i) { + return LOAD_TESTS_KEY_GENERATOR.generate(i); + } + + public static Object generateLoadTestsValue(int i) { + return LOAD_TESTS_VALUE_GENERATOR.generate(i); + } + + @SuppressWarnings("unchecked") + public static CacheEntryImpl generateLoadTestsEntry(int i) { + return new CacheEntryImpl(TestsHelper.generateLoadTestsKey(i), TestsHelper.generateLoadTestsValue(i)); + } + + public static Collection getKeys(Collection> entries) { + List list = new LinkedList<>(); + + for (CacheEntryImpl entry : entries) + list.add(entry.getKey()); + + return list; + } + + public static Map generateIntegersMap() { + return generateIntegersMap(BULK_OPERATION_SIZE); + } + + public static Map generateIntegersMap(int count) { + Map map = new HashMap<>(); + + for (int i = 0; i < count; i++) + map.put(i, i + 123); + + return map; + } + + public static Collection> generateIntegersEntries() { + return generateIntegersEntries(BULK_OPERATION_SIZE); + } + + public static Collection> generateIntegersEntries(int count) { + Collection> entries = new LinkedList<>(); + + for (int i = 0; i < count; i++) + entries.add(new CacheEntryImpl<>(i, i + 123)); + + return entries; + } + + public static Map generateStringsMap() { + return generateStringsMap(BULK_OPERATION_SIZE); + } + + public static Map generateStringsMap(int count) { + Map map = new HashMap<>(); + + for (int i = 0; i < count; i++) + map.put(Integer.toString(i), randomString(5)); + + return map; + } + + public static Collection> generateStringsEntries() { + return generateStringsEntries(BULK_OPERATION_SIZE); + } + + public static Collection> generateStringsEntries(int count) { + Collection> entries = new LinkedList<>(); + + for (int i = 0; i < count; i++) + entries.add(new CacheEntryImpl<>(Integer.toString(i), randomString(5))); + + return entries; + } + + public static Map generateIntegersPersonsMap() { + Map map = new HashMap<>(); + + for (int i = 0; i < BULK_OPERATION_SIZE; i++) + map.put(i, generateRandomPerson()); + + return map; + } + + public static Collection> generateIntegersPersonsEntries() { + Collection> entries = new LinkedList<>(); + + for (int i = 0; i < BULK_OPERATION_SIZE; i++) + entries.add(new CacheEntryImpl<>(i, generateRandomPerson())); + + return entries; + } + + public static Map generatePersonIdsPersonsMap() { + return generatePersonIdsPersonsMap(BULK_OPERATION_SIZE); + } + + public static Map generatePersonIdsPersonsMap(int count) { + Map map = new HashMap<>(); + + for (int i = 0; i < count; i++) + map.put(generateRandomPersonId(), generateRandomPerson()); + + return map; + } + + public static Collection> generatePersonIdsPersonsEntries() { + return generatePersonIdsPersonsEntries(BULK_OPERATION_SIZE); + } + + public static Collection> generatePersonIdsPersonsEntries(int count) { + Collection> entries = new LinkedList<>(); + + for (int i = 0; i < count; i++) + entries.add(new CacheEntryImpl<>(generateRandomPersonId(), generateRandomPerson())); + + return entries; + } + + public static Person generateRandomPerson() { + int phonesCount = RANDOM.nextInt(4); + + List phones = new LinkedList<>(); + + for (int i = 0; i < phonesCount; i++) { + phones.add(randomNumber(4)); + } + + return new Person(randomString(4), randomString(4), RANDOM.nextInt(100), + RANDOM.nextBoolean(), RANDOM.nextLong(), RANDOM.nextFloat(), new Date(), phones); + } + + public static PersonId generateRandomPersonId() { + return new PersonId(randomString(4), randomString(4), RANDOM.nextInt(100)); + } + + public static boolean checkMapsEqual(Map map1, Map map2) { + if (map1 == null || map2 == null || map1.size() != map2.size()) + return false; + + for (Object key : map1.keySet()) { + Object obj1 = map1.get(key); + Object obj2 = map2.get(key); + + if (obj1 == null || obj2 == null || !obj1.equals(obj2)) + return false; + } + + return true; + } + + public static boolean checkCollectionsEqual(Map map, Collection> collection) { + if (map == null || collection == null || map.size() != collection.size()) + return false; + + for (CacheEntryImpl entry : collection) { + if (!entry.getValue().equals(map.get(entry.getKey()))) + return false; + } + + return true; + } + + public static boolean checkPersonMapsEqual(Map map1, Map map2, + boolean primitiveFieldsOnly) { + if (map1 == null || map2 == null || map1.size() != map2.size()) + return false; + + for (K key : map1.keySet()) { + Person person1 = map1.get(key); + Person person2 = map2.get(key); + + boolean equals = person1 != null && person2 != null && + (primitiveFieldsOnly ? person1.equalsPrimitiveFields(person2) : person1.equals(person2)); + + if (!equals) + return false; + } + + return true; + } + + public static boolean checkPersonCollectionsEqual(Map map, + Collection> collection, boolean primitiveFieldsOnly) { + if (map == null || collection == null || map.size() != collection.size()) + return false; + + for (CacheEntryImpl entry : collection) { + boolean equals = primitiveFieldsOnly ? + entry.getValue().equalsPrimitiveFields(map.get(entry.getKey())) : + entry.getValue().equals(map.get(entry.getKey())); + + if (!equals) + return false; + } + + return true; + } + + public static String randomString(int length) { + StringBuilder builder = new StringBuilder(length); + + for (int i = 0; i < length; i++) + builder.append(LETTERS_ALPHABET.charAt(RANDOM.nextInt(LETTERS_ALPHABET.length()))); + + return builder.toString(); + } + + public static String randomNumber(int length) { + StringBuilder builder = new StringBuilder(length); + + for (int i = 0; i < length; i++) + builder.append(NUMBERS_ALPHABET.charAt(RANDOM.nextInt(NUMBERS_ALPHABET.length()))); + + return builder.toString(); + } +} diff --git a/modules/cassandra/src/test/resources/log4j.properties b/modules/cassandra/src/test/resources/log4j.properties new file mode 100644 index 0000000..bcfadf9 --- /dev/null +++ b/modules/cassandra/src/test/resources/log4j.properties @@ -0,0 +1,104 @@ +# Root logger option +log4j.rootLogger=INFO, stdout + +# Direct log messages to stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +# ------ LOAD TESTS LOGGING ------ + +# Cassandra load tests loggers +log4j.logger.CassandraLoadTests=INFO, stdout, cassandraLoadTests +log4j.logger.CassandraWriteLoadTest=INFO, csWrite +log4j.logger.CassandraBulkWriteLoadTest=INFO, csBulkWrite +log4j.logger.CassandraReadLoadTest=INFO, csRead +log4j.logger.CassandraBulkReadLoadTest=INFO, csBulkRead + +log4j.appender.cassandraLoadTests=org.apache.log4j.RollingFileAppender +log4j.appender.cassandraLoadTests.File=logs/cassandra-load-tests.log +log4j.appender.cassandraLoadTests.MaxFileSize=10MB +log4j.appender.cassandraLoadTests.MaxBackupIndex=10 +log4j.appender.cassandraLoadTests.Append=true +log4j.appender.cassandraLoadTests.layout=org.apache.log4j.PatternLayout +log4j.appender.cassandraLoadTests.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.csWrite=org.apache.log4j.RollingFileAppender +log4j.appender.csWrite.File=logs/cassandra-write.log +log4j.appender.csWrite.MaxFileSize=10MB +log4j.appender.csWrite.MaxBackupIndex=10 +log4j.appender.csWrite.Append=true +log4j.appender.csWrite.layout=org.apache.log4j.PatternLayout +log4j.appender.csWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.csBulkWrite=org.apache.log4j.RollingFileAppender +log4j.appender.csBulkWrite.File=logs/cassandra-bulk-write.log +log4j.appender.csBulkWrite.MaxFileSize=10MB +log4j.appender.csBulkWrite.MaxBackupIndex=10 +log4j.appender.csBulkWrite.Append=true +log4j.appender.csBulkWrite.layout=org.apache.log4j.PatternLayout +log4j.appender.csBulkWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.csRead=org.apache.log4j.RollingFileAppender +log4j.appender.csRead.File=logs/cassandra-read.log +log4j.appender.csRead.MaxFileSize=10MB +log4j.appender.csRead.MaxBackupIndex=10 +log4j.appender.csRead.Append=true +log4j.appender.csRead.layout=org.apache.log4j.PatternLayout +log4j.appender.csRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.csBulkRead=org.apache.log4j.RollingFileAppender +log4j.appender.csBulkRead.File=logs/cassandra-bulk-read.log +log4j.appender.csBulkRead.MaxFileSize=10MB +log4j.appender.csBulkRead.MaxBackupIndex=10 +log4j.appender.csBulkRead.Append=true +log4j.appender.csBulkRead.layout=org.apache.log4j.PatternLayout +log4j.appender.csBulkRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +# Ignite load tests loggers +log4j.logger.IgniteLoadTests=INFO, stdout, igniteLoadTests +log4j.logger.IgniteWriteLoadTest=INFO, igWrite +log4j.logger.IgniteBulkWriteLoadTest=INFO, igBulkWrite +log4j.logger.IgniteReadLoadTest=INFO, igRead +log4j.logger.IgniteBulkReadLoadTest=INFO, igBulkRead + +log4j.appender.igniteLoadTests=org.apache.log4j.RollingFileAppender +log4j.appender.igniteLoadTests.File=logs/ignite-load-tests.log +log4j.appender.igniteLoadTests.MaxFileSize=10MB +log4j.appender.igniteLoadTests.MaxBackupIndex=10 +log4j.appender.igniteLoadTests.Append=true +log4j.appender.igniteLoadTests.layout=org.apache.log4j.PatternLayout +log4j.appender.igniteLoadTests.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.igWrite=org.apache.log4j.RollingFileAppender +log4j.appender.igWrite.File=logs/ignite-write.log +log4j.appender.igWrite.MaxFileSize=10MB +log4j.appender.igWrite.MaxBackupIndex=10 +log4j.appender.igWrite.Append=true +log4j.appender.igWrite.layout=org.apache.log4j.PatternLayout +log4j.appender.igWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.igBulkWrite=org.apache.log4j.RollingFileAppender +log4j.appender.igBulkWrite.File=logs/ignite-bulk-write.log +log4j.appender.igBulkWrite.MaxFileSize=10MB +log4j.appender.igBulkWrite.MaxBackupIndex=10 +log4j.appender.igBulkWrite.Append=true +log4j.appender.igBulkWrite.layout=org.apache.log4j.PatternLayout +log4j.appender.igBulkWrite.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.igRead=org.apache.log4j.RollingFileAppender +log4j.appender.igRead.File=logs/ignite-read.log +log4j.appender.igRead.MaxFileSize=10MB +log4j.appender.igRead.MaxBackupIndex=10 +log4j.appender.igRead.Append=true +log4j.appender.igRead.layout=org.apache.log4j.PatternLayout +log4j.appender.igRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n + +log4j.appender.igBulkRead=org.apache.log4j.RollingFileAppender +log4j.appender.igBulkRead.File=logs/ignite-bulk-read.log +log4j.appender.igBulkRead.MaxFileSize=10MB +log4j.appender.igBulkRead.MaxBackupIndex=10 +log4j.appender.igBulkRead.Append=true +log4j.appender.igBulkRead.layout=org.apache.log4j.PatternLayout +log4j.appender.igBulkRead.layout.ConversionPattern=%d{HH:mm:ss,SSS} %5p [%t] - %m%n diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml new file mode 100644 index 0000000..4ddc38a --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection-settings.xml @@ -0,0 +1,30 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties new file mode 100644 index 0000000..f8d44ca --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/connection.properties @@ -0,0 +1,2 @@ +# Comma delimited Cassandra contact points in format: host[:port] +contact.points=my-server-1.test.com,my-server-2.test.com \ No newline at end of file diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties new file mode 100644 index 0000000..ede97e5 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/credentials.properties @@ -0,0 +1,7 @@ +# Cassandra admin user/password +admin.user= +admin.password= + +# Cassandra regular user/password +regular.user= +regular.password= \ No newline at end of file diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties new file mode 100644 index 0000000..6dc5e34 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/cassandra/keyspaces.properties @@ -0,0 +1,2 @@ +# Cassandra keyspaces used for tests +keyspaces=test1 \ No newline at end of file diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml new file mode 100644 index 0000000..fb35ba5 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/ignite-config.xml @@ -0,0 +1,82 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml new file mode 100644 index 0000000..55b4c1b --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-1.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml new file mode 100644 index 0000000..32c1d9e --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-2.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml new file mode 100644 index 0000000..64a9f95 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/blob/persistence-settings-3.xml @@ -0,0 +1,12 @@ + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml new file mode 100644 index 0000000..9e74951 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/ignite-config.xml @@ -0,0 +1,101 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml new file mode 100644 index 0000000..98653a8 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-1.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml new file mode 100644 index 0000000..34f83b8 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-2.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml new file mode 100644 index 0000000..f69a3b2 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/pojo/persistence-settings-3.xml @@ -0,0 +1,31 @@ + + + REPLICATION = {'class' : 'SimpleStrategy', 'replication_factor' : 3} + AND DURABLE_WRITES = true + + + comment = 'A most excellent and useful table' + AND read_repair_chance = 0.2 + + + + + + + + + + + + + + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml new file mode 100644 index 0000000..e350ded --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-config.xml @@ -0,0 +1,82 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml new file mode 100644 index 0000000..0064727 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml @@ -0,0 +1,75 @@ + + + + + + + ecsa001005b9.epam.com + ecsa001005bd.epam.com + + + + + + + + + + + + + + + + +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ECSA001005C5 + ECSA001005C7 + ECSA001005C6 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml new file mode 100644 index 0000000..d7ac277 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/ignite-remote-server-config.xml @@ -0,0 +1,88 @@ + + + + + + + + ecsa001005b9.epam.com + ecsa001005bd.epam.com + + + + + + + + + + + + + + + + + +]]> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + 127.0.0.1:47500..47509 + + + + + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml new file mode 100644 index 0000000..f196fb6 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml new file mode 100644 index 0000000..eaed7e5 --- /dev/null +++ b/modules/cassandra/src/test/resources/org/apache/ignite/tests/persistence/primitive/persistence-settings-2.xml @@ -0,0 +1,4 @@ + + + + diff --git a/modules/cassandra/src/test/resources/tests.properties b/modules/cassandra/src/test/resources/tests.properties new file mode 100644 index 0000000..10b71a8 --- /dev/null +++ b/modules/cassandra/src/test/resources/tests.properties @@ -0,0 +1,31 @@ +# Number of elements for CacheStore bulk operations: loadAll, writeAll, deleteAll +bulk.operation.size=100 + +# ----- Load tests settings ----- + +# Ignite cache to be used by load tests +load.tests.cache.name=cache1 + +# Number of simultaneous threads for each load test +load.tests.threads.count=5 + +# Warm up period (in milliseconds) for each load test before starting any measurements +load.tests.warmup.period=120000 + +# Whole time for each load test execution (in milliseconds) +load.tests.execution.time=300000 + +# Latency (in milliseconds) between two sequential requests to Cassandra/Ignite +load.tests.requests.latency=0 + +# Resource specifying persistence settings for all load tests +load.tests.persistence.settings=org/apache/ignite/tests/persistence/primitive/persistence-settings-1.xml + +# Resource specifying Ignite configuration for all load tests +load.tests.ignite.config=org/apache/ignite/tests/persistence/primitive/ignite-remote-client-config.xml + +# Key generator for load tests +load.tests.key.generator=org.apache.ignite.tests.load.IntGenerator + +# Value generator for load tests +load.tests.value.generator=org.apache.ignite.tests.load.IntGenerator \ No newline at end of file diff --git a/modules/cassandra/src/test/scripts/cassandra-load-tests.bat b/modules/cassandra/src/test/scripts/cassandra-load-tests.bat new file mode 100644 index 0000000..0042ac4 --- /dev/null +++ b/modules/cassandra/src/test/scripts/cassandra-load-tests.bat @@ -0,0 +1,37 @@ +echo off + +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +echo. + +set TESTS_CLASSPATH="%~dp0lib\*;%~dp0settings" + +call java -cp "%TESTS_CLASSPATH%" "org.apache.ignite.tests.CassandraDirectPersistenceLoadTest" + +if %errorLevel% NEQ 0 ( + echo. + echo -------------------------------------------------------------------------------- + echo [ERROR] Tests execution failed + echo -------------------------------------------------------------------------------- + echo. + exit /b %errorLevel% +) + +echo. +echo -------------------------------------------------------------------------------- +echo [INFO] Tests execution succeed +echo -------------------------------------------------------------------------------- +echo. diff --git a/modules/cassandra/src/test/scripts/cassandra-load-tests.sh b/modules/cassandra/src/test/scripts/cassandra-load-tests.sh new file mode 100644 index 0000000..59d55de --- /dev/null +++ b/modules/cassandra/src/test/scripts/cassandra-load-tests.sh @@ -0,0 +1,37 @@ +#!/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_ROOT=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )) +TESTS_CLASSPATH="$TESTS_ROOT/lib/*:$TESTS_ROOT/settings" + +java -cp "$TESTS_CLASSPATH" "org.apache.ignite.tests.CassandraDirectPersistenceLoadTest" + +if [ $? -ne 0 ]; then + echo + echo "--------------------------------------------------------------------------------" + echo "[ERROR] Tests execution failed" + echo "--------------------------------------------------------------------------------" + echo + exit 1 +fi + +echo +echo "--------------------------------------------------------------------------------" +echo "[INFO] Tests execution succeed" +echo "--------------------------------------------------------------------------------" +echo diff --git a/modules/cassandra/src/test/scripts/ignite-load-tests.bat b/modules/cassandra/src/test/scripts/ignite-load-tests.bat new file mode 100644 index 0000000..c11bac5 --- /dev/null +++ b/modules/cassandra/src/test/scripts/ignite-load-tests.bat @@ -0,0 +1,37 @@ +echo off + +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +echo. + +set TESTS_CLASSPATH="%~dp0\lib*;%~dp0settings" + +call java -cp "%TESTS_CLASSPATH%" "org.apache.ignite.tests.IgnitePersistentStoreLoadTest" + +if %errorLevel% NEQ 0 ( + echo. + echo -------------------------------------------------------------------------------- + echo [ERROR] Tests execution failed + echo -------------------------------------------------------------------------------- + echo. + exit /b %errorLevel% +) + +echo. +echo -------------------------------------------------------------------------------- +echo [INFO] Tests execution succeed +echo -------------------------------------------------------------------------------- +echo. diff --git a/modules/cassandra/src/test/scripts/ignite-load-tests.sh b/modules/cassandra/src/test/scripts/ignite-load-tests.sh new file mode 100644 index 0000000..2b0364b --- /dev/null +++ b/modules/cassandra/src/test/scripts/ignite-load-tests.sh @@ -0,0 +1,37 @@ +#!/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_ROOT=$(readlink -m $( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )) +TESTS_CLASSPATH="$TESTS_ROOT/lib/*:$TESTS_ROOT/settings" + +java -cp "$TESTS_CLASSPATH" "org.apache.ignite.tests.IgnitePersistentStoreLoadTest" + +if [ $? -ne 0 ]; then + echo + echo "--------------------------------------------------------------------------------" + echo "[ERROR] Tests execution failed" + echo "--------------------------------------------------------------------------------" + echo + exit 1 +fi + +echo +echo "--------------------------------------------------------------------------------" +echo "[INFO] Tests execution succeed" +echo "--------------------------------------------------------------------------------" +echo diff --git a/pom.xml b/pom.xml index 7b8763f..c0ff91f 100644 --- a/pom.xml +++ b/pom.xml @@ -78,6 +78,7 @@ modules/mqtt modules/zookeeper modules/platform + modules/cassandra -- 2.2.2