diff --git a/.travis.yml b/.travis.yml index 6fe0802652..ade1445119 100644 --- a/.travis.yml +++ b/.travis.yml @@ -39,6 +39,7 @@ env: - MODULE='hbase-parent/janusgraph-hbase-10' ARGS='-Dtest=**/graphdb/hbase/*' - MODULE='hbase-parent/janusgraph-hbase-098' ARGS='-Dtest=**/diskstorage/hbase/*' - MODULE='hbase-parent/janusgraph-hbase-098' ARGS='-Dtest=**/graphdb/hbase/*' + - MODULE='cql' matrix: fast_finish: true @@ -47,6 +48,7 @@ matrix: # Can fail due to timeout (runs longer than 50min) - env: MODULE='hbase-parent/janusgraph-hbase-098' ARGS='-Dtest=**/diskstorage/hbase/*' - env: MODULE='hbase-parent/janusgraph-hbase-098' ARGS='-Dtest=**/graphdb/hbase/*' + - env: MODULE='cql' addons: coverity_scan: diff --git a/AUTHORS.txt b/AUTHORS.txt index 70c5de6ff3..9b6573ad95 100644 --- a/AUTHORS.txt +++ b/AUTHORS.txt @@ -16,5 +16,6 @@ DataStax Dylan Bethune-Waddell Expero Google +Orion Health IBM Rafael Fernandes diff --git a/CONTRIBUTORS.txt b/CONTRIBUTORS.txt index b9f4c94398..ac4d1141dd 100644 --- a/CONTRIBUTORS.txt +++ b/CONTRIBUTORS.txt @@ -74,6 +74,7 @@ Mike McMahon Misha Brukman MrKeyholder Nik Everett +Paul Kendall Pavel Yaskevich Peter Beaman Petter Aas @@ -82,6 +83,7 @@ PommeVerte Rafael Fernandes Ranger Tsao Richard Doll +Samant Maharaj Sebastian Good Solon Gordon Stephen Mallette diff --git a/janusgraph-all/pom.xml b/janusgraph-all/pom.xml index 89341b2fba..a4c1c3c5cb 100644 --- a/janusgraph-all/pom.xml +++ b/janusgraph-all/pom.xml @@ -33,6 +33,11 @@ janusgraph-cassandra ${project.version} + + org.janusgraph + janusgraph-cql + ${project.version} + org.janusgraph janusgraph-berkeleyje diff --git a/janusgraph-core/src/main/java/org/janusgraph/core/util/ReflectiveConfigOptionLoader.java b/janusgraph-core/src/main/java/org/janusgraph/core/util/ReflectiveConfigOptionLoader.java index ed5d625e05..c56576a1a7 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/core/util/ReflectiveConfigOptionLoader.java +++ b/janusgraph-core/src/main/java/org/janusgraph/core/util/ReflectiveConfigOptionLoader.java @@ -121,6 +121,7 @@ public void loadStandard(Class caller) { "org.janusgraph.diskstorage.cassandra.astyanax.AstyanaxStoreManager", "org.janusgraph.diskstorage.cassandra.AbstractCassandraStoreManager", "org.janusgraph.diskstorage.cassandra.thrift.CassandraThriftStoreManager", + "org.janusgraph.diskstorage.cql.CQLConfigOptions", "org.janusgraph.diskstorage.es.ElasticSearchIndex", "org.janusgraph.diskstorage.solr.SolrIndex", "org.janusgraph.diskstorage.log.kcvs.KCVSLog", diff --git a/janusgraph-core/src/main/java/org/janusgraph/diskstorage/StandardStoreManager.java b/janusgraph-core/src/main/java/org/janusgraph/diskstorage/StandardStoreManager.java index df35493c0e..b0d95c35ed 100644 --- a/janusgraph-core/src/main/java/org/janusgraph/diskstorage/StandardStoreManager.java +++ b/janusgraph-core/src/main/java/org/janusgraph/diskstorage/StandardStoreManager.java @@ -28,6 +28,7 @@ public enum StandardStoreManager { CASSANDRA_THRIFT("org.janusgraph.diskstorage.cassandra.thrift.CassandraThriftStoreManager", "cassandrathrift"), CASSANDRA_ASTYANAX("org.janusgraph.diskstorage.cassandra.astyanax.AstyanaxStoreManager", ImmutableList.of("cassandra", "astyanax")), CASSANDRA_EMBEDDED("org.janusgraph.diskstorage.cassandra.embedded.CassandraEmbeddedStoreManager", "embeddedcassandra"), + CQL("org.janusgraph.diskstorage.cql.CQLStoreManager", "cql"), HBASE("org.janusgraph.diskstorage.hbase.HBaseStoreManager", "hbase"), IN_MEMORY("org.janusgraph.diskstorage.keycolumnvalue.inmemory.InMemoryStoreManager", "inmemory"); diff --git a/janusgraph-cql/pom.xml b/janusgraph-cql/pom.xml new file mode 100644 index 0000000000..aef2f57e9f --- /dev/null +++ b/janusgraph-cql/pom.xml @@ -0,0 +1,314 @@ + + 4.0.0 + + + org.janusgraph + janusgraph + 0.2.0-SNAPSHOT + + + janusgraph-cql + JanusGraph-CQL: Distributed Graph Database + http://janusgraph.org + + + 3.1.4 + 2.0.5 + + byteorderedpartitioner + murmur + murmur-ssl + -Xms256m -Xmx1280m -ea -XX:+HeapDumpOnOutOfMemoryError -javaagent:${com.github.jbellis:jamm:jar} + org.janusgraph.testcategory.MemoryTests,org.janusgraph.testcategory.PerformanceTests,org.janusgraph.testcategory.BrittleTests + + + + + + org.janusgraph + janusgraph-core + ${project.version} + + + + com.datastax.cassandra + cassandra-driver-core + ${cassandra-driver.version} + + + com.google.guava + guava + + + io.javaslang + javaslang + ${javaslang.version} + + + + + org.slf4j + slf4j-log4j12 + test + + + org.janusgraph + janusgraph-cassandra + ${project.version} + test + + + org.janusgraph + janusgraph-cassandra + ${project.version} + tests + test + + + org.janusgraph + janusgraph-test + ${project.version} + test + + + junit + junit + test + + + org.mockito + mockito-all + test + + + + + + + src/test/resources + + cassandra/ + keystores/ + properties/ + + true + + + + + maven-dependency-plugin + + + set-dependency-properties + process-test-classes + + properties + + + + + + maven-resources-plugin + + + filter-${test.byteorderedpartitioner} + process-test-resources + + copy-resources + + + ${project.build.directory}/cassandra/${test.byteorderedpartitioner}/conf + + ${project.build.testResources[0].directory}/properties/${test.byteorderedpartitioner}.properties + + + + ${project.build.testResources[0].directory}/cassandra + true + + + + + + filter-${test.murmur} + process-test-resources + + copy-resources + + + ${project.build.directory}/cassandra/${test.murmur}/conf + + ${project.build.testResources[0].directory}/properties/${test.murmur}.properties + + + + ${project.build.testResources[0].directory}/cassandra + true + + + + + + filter-${test.murmur-ssl} + process-test-resources + + copy-resources + + + ${project.build.directory}/cassandra/${test.murmur-ssl}/conf + + ${project.build.testResources[0].directory}/properties/${test.murmur-ssl}.properties + + + + ${project.build.testResources[0].directory}/cassandra + true + + + + + + filter-${test.murmur-ssl}-static + process-test-resources + + copy-resources + + + ${project.build.directory}/cassandra/${test.murmur-ssl}/conf + + + ${project.build.testResources[0].directory}/keystores + false + + + + + + + + maven-surefire-plugin + + ${test.jvm.opts} + 1 + false + classes + true + 2 + random + + file:${project.build.testOutputDirectory}/log4j.properties + + + + + default-test + none + + + ${test.byteorderedpartitioner}-test + + test + + test + + ${test.excluded.groups},org.janusgraph.testcategory.UnorderedKeyStoreTests,org.janusgraph.testcategory.SerialTests,org.janusgraph.testcategory.CassandraSSLTests + + ${test.byteorderedpartitioner} + + + test.cassandra.confdir + ${project.build.directory}/cassandra/${test.byteorderedpartitioner}/conf + + + test.cassandra.datadir + ${project.build.directory}/cassandra/${test.byteorderedpartitioner}/data + + + ${test.skip.byteorderedpartitioner} + + + + ${test.murmur}-test + + test + + test + + ${test.excluded.groups},org.janusgraph.testcategory.OrderedKeyStoreTests,org.janusgraph.testcategory.SerialTests,org.janusgraph.testcategory.CassandraSSLTests + + ${test.murmur} + + + test.cassandra.confdir + ${project.build.directory}/cassandra/${test.murmur}/conf + + + test.cassandra.datadir + ${project.build.directory}/cassandra/${test.murmur}/data + + + ${test.skip.murmur} + + + + ${test.murmur}-serial-test + + test + + test + + + org.janusgraph.testcategory.SerialTests + none + false + 1 + alphabetical + ${test.murmur}-serial + + + test.cassandra.confdir + ${project.build.directory}/cassandra/${test.murmur}/conf + + + test.cassandra.datadir + ${project.build.directory}/cassandra/${test.murmur}/data + + + ${test.skip.murmur-serial} + + + + ${test.murmur-ssl}-test + + test + + test + + + org.janusgraph.testcategory.CassandraSSLTests + + + **/*ComputerTest.java + **/*StructureTest.java + **/*ProcessTest.java + + 1 + ${test.murmur-ssl} + + + test.cassandra.confdir + ${project.build.directory}/cassandra/${test.murmur-ssl}/conf + + + test.cassandra.datadir + ${project.build.directory}/cassandra/${test.murmur-ssl}/data + + + ${test.skip.murmur-ssl} + + + + + + + diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLColValGetter.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLColValGetter.java new file mode 100644 index 0000000000..a55355fda2 --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLColValGetter.java @@ -0,0 +1,59 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import org.janusgraph.diskstorage.EntryMetaData; +import org.janusgraph.diskstorage.StaticBuffer; +import org.janusgraph.diskstorage.util.StaticArrayEntry.GetColVal; + +import com.datastax.driver.core.Row; + +import javaslang.Tuple3; + +class CQLColValGetter implements GetColVal, StaticBuffer> { + + private final EntryMetaData[] schema; + + CQLColValGetter(final EntryMetaData[] schema) { + this.schema = schema; + } + + @Override + public StaticBuffer getColumn(final Tuple3 tuple) { + return tuple._1; + } + + @Override + public StaticBuffer getValue(final Tuple3 tuple) { + return tuple._2; + } + + @Override + public EntryMetaData[] getMetaSchema(final Tuple3 tuple) { + return this.schema; + } + + @Override + public Object getMetaData(final Tuple3 tuple, final EntryMetaData metaData) { + switch (metaData) { + case TIMESTAMP: + return tuple._3.getLong(CQLKeyColumnValueStore.WRITETIME_COLUMN_NAME); + case TTL: + return tuple._3.getInt(CQLKeyColumnValueStore.TTL_COLUMN_NAME); + default: + throw new UnsupportedOperationException("Unsupported meta data: " + metaData); + } + } +} \ No newline at end of file diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLConfigOptions.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLConfigOptions.java new file mode 100644 index 0000000000..4191c48320 --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLConfigOptions.java @@ -0,0 +1,194 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import org.janusgraph.diskstorage.configuration.ConfigElement; +import org.janusgraph.diskstorage.configuration.ConfigNamespace; +import org.janusgraph.diskstorage.configuration.ConfigOption; +import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; +import org.janusgraph.graphdb.configuration.PreInitializeConfigOptions; + +/** + * Configuration options for the CQL storage backend. These are managed under the 'cql' namespace in the configuration. + */ +@PreInitializeConfigOptions +public interface CQLConfigOptions { + + public static final ConfigNamespace CQL_NS = new ConfigNamespace( + GraphDatabaseConfiguration.STORAGE_NS, + "cql", + "CQL storage backend options"); + + public static final ConfigOption KEYSPACE = new ConfigOption<>( + CQL_NS, + "keyspace", + "The name of JanusGraph's keyspace. It will be created if it does not exist.", + ConfigOption.Type.LOCAL, + "janusgraph"); + + public static final ConfigOption PROTOCOL_VERSION = new ConfigOption<>( + CQL_NS, + "protocol-version", + "The protocol version used to connect to the Cassandra database. If no value is supplied then the driver will negotiate with the server.", + ConfigOption.Type.LOCAL, + 0); + + public static final ConfigOption READ_CONSISTENCY = new ConfigOption<>( + CQL_NS, + "read-consistency-level", + "The consistency level of read operations against Cassandra", + ConfigOption.Type.MASKABLE, + CQLStoreManager.CONSISTENCY_QUORUM); + + public static final ConfigOption WRITE_CONSISTENCY = new ConfigOption<>( + CQL_NS, + "write-consistency-level", + "The consistency level of write operations against Cassandra", + ConfigOption.Type.MASKABLE, + CQLStoreManager.CONSISTENCY_QUORUM); + + // The number of statements in a batch + public static final ConfigOption BATCH_STATEMENT_SIZE = new ConfigOption<>( + CQL_NS, + "batch-statement-size", + "The number of statements in each batch", + ConfigOption.Type.MASKABLE, + 20); + + // Whether to use unlogged batches + public static final ConfigOption ATOMIC_BATCH_MUTATE = new ConfigOption<>( + CQL_NS, + "atomic-batch-mutate", + "True to use Cassandra atomic batch mutation, false to use non-atomic batches", + ConfigOption.Type.MASKABLE, + false); + + // Replication + public static final ConfigOption REPLICATION_FACTOR = new ConfigOption<>( + CQL_NS, + "replication-factor", + "The number of data replicas (including the original copy) that should be kept", + ConfigOption.Type.GLOBAL_OFFLINE, + 1); + + public static final ConfigOption REPLICATION_STRATEGY = new ConfigOption<>( + CQL_NS, + "replication-strategy-class", + "The replication strategy to use for JanusGraph keyspace", + ConfigOption.Type.FIXED, + "SimpleStrategy"); + + public static final ConfigOption REPLICATION_OPTIONS = new ConfigOption<>( + CQL_NS, + "replication-strategy-options", + "Replication strategy options, e.g. factor or replicas per datacenter. This list is interpreted as a " + + "map. It must have an even number of elements in [key,val,key,val,...] form. A replication_factor set " + + "here takes precedence over one set with " + ConfigElement.getPath(REPLICATION_FACTOR), + ConfigOption.Type.FIXED, + String[].class); + + public static final ConfigOption COMPACTION_STRATEGY = new ConfigOption<>( + CQL_NS, + "compaction-strategy-class", + "The compaction strategy to use for JanusGraph tables", + ConfigOption.Type.FIXED, + String.class); + + public static final ConfigOption COMPACTION_OPTIONS = new ConfigOption<>( + CQL_NS, + "compaction-strategy-options", + "Compaction strategy options. This list is interpreted as a " + + "map. It must have an even number of elements in [key,val,key,val,...] form.", + ConfigOption.Type.FIXED, + String[].class); + + // Compression + public static final ConfigOption CF_COMPRESSION = new ConfigOption<>( + CQL_NS, + "compression", + "Whether the storage backend should use compression when storing the data", + ConfigOption.Type.FIXED, + true); + + public static final ConfigOption CF_COMPRESSION_TYPE = new ConfigOption<>( + CQL_NS, + "compression-type", + "The sstable_compression value JanusGraph uses when creating column families. " + + "This accepts any value allowed by Cassandra's sstable_compression option. " + + "Leave this unset to disable sstable_compression on JanusGraph-created CFs.", + ConfigOption.Type.MASKABLE, + "LZ4Compressor"); + + public static final ConfigOption CF_COMPRESSION_BLOCK_SIZE = new ConfigOption<>( + CQL_NS, + "compression-block-size", + "The size of the compression blocks in kilobytes", + ConfigOption.Type.FIXED, + 64); + + // SSL + public static final ConfigNamespace SSL_NS = new ConfigNamespace( + CQL_NS, + "ssl", + "Configuration options for SSL"); + + public static final ConfigNamespace SSL_TRUSTSTORE_NS = new ConfigNamespace( + SSL_NS, + "truststore", + "Configuration options for SSL Truststore."); + + public static final ConfigOption SSL_ENABLED = new ConfigOption<>( + SSL_NS, + "enabled", + "Controls use of the SSL connection to Cassandra", + ConfigOption.Type.LOCAL, + false); + + public static final ConfigOption SSL_TRUSTSTORE_LOCATION = new ConfigOption<>( + SSL_TRUSTSTORE_NS, + "location", + "Marks the location of the SSL Truststore.", + ConfigOption.Type.LOCAL, + ""); + + public static final ConfigOption SSL_TRUSTSTORE_PASSWORD = new ConfigOption<>( + SSL_TRUSTSTORE_NS, + "password", + "The password to access SSL Truststore.", + ConfigOption.Type.LOCAL, + ""); + + // Other options + public static final ConfigOption CLUSTER_NAME = new ConfigOption<>( + CQL_NS, + "cluster-name", + "Default name for the Cassandra cluster", + ConfigOption.Type.MASKABLE, + "JanusGraph Cluster"); + + public static final ConfigOption LOCAL_DATACENTER = new ConfigOption<>( + CQL_NS, + "local-datacenter", + "The name of the local or closest Cassandra datacenter. When set and not whitespace, " + + "this value will be passed into ConnectionPoolConfigurationImpl.setLocalDatacenter. " + + "When unset or set to whitespace, setLocalDatacenter will not be invoked.", + /* + * It's between either LOCAL or MASKABLE. MASKABLE could be useful for cases where all the JanusGraph instances are closest to + * the same Cassandra DC. + */ + ConfigOption.Type.MASKABLE, + String.class); + +} diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLKeyColumnValueStore.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLKeyColumnValueStore.java new file mode 100644 index 0000000000..e68eda7eb7 --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLKeyColumnValueStore.java @@ -0,0 +1,397 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static com.datastax.driver.core.querybuilder.QueryBuilder.bindMarker; +import static com.datastax.driver.core.querybuilder.QueryBuilder.column; +import static com.datastax.driver.core.querybuilder.QueryBuilder.delete; +import static com.datastax.driver.core.querybuilder.QueryBuilder.eq; +import static com.datastax.driver.core.querybuilder.QueryBuilder.gte; +import static com.datastax.driver.core.querybuilder.QueryBuilder.insertInto; +import static com.datastax.driver.core.querybuilder.QueryBuilder.lt; +import static com.datastax.driver.core.querybuilder.QueryBuilder.lte; +import static com.datastax.driver.core.querybuilder.QueryBuilder.select; +import static com.datastax.driver.core.querybuilder.QueryBuilder.timestamp; +import static com.datastax.driver.core.querybuilder.QueryBuilder.token; +import static com.datastax.driver.core.querybuilder.QueryBuilder.ttl; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.createTable; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.dateTieredStrategy; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.deflate; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.leveledStrategy; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.lz4; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.noCompression; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.sizedTieredStategy; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.snappy; +import static javaslang.API.$; +import static javaslang.API.Case; +import static javaslang.API.Match; +import static javaslang.Predicates.instanceOf; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.CF_COMPRESSION; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.CF_COMPRESSION_BLOCK_SIZE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.CF_COMPRESSION_TYPE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.COMPACTION_OPTIONS; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.COMPACTION_STRATEGY; +import static org.janusgraph.diskstorage.cql.CQLTransaction.getTransaction; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.function.Function; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.Entry; +import org.janusgraph.diskstorage.EntryList; +import org.janusgraph.diskstorage.EntryMetaData; +import org.janusgraph.diskstorage.PermanentBackendException; +import org.janusgraph.diskstorage.StaticBuffer; +import org.janusgraph.diskstorage.TemporaryBackendException; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.keycolumnvalue.KCVMutation; +import org.janusgraph.diskstorage.keycolumnvalue.KeyColumnValueStore; +import org.janusgraph.diskstorage.keycolumnvalue.KeyIterator; +import org.janusgraph.diskstorage.keycolumnvalue.KeyRangeQuery; +import org.janusgraph.diskstorage.keycolumnvalue.KeySliceQuery; +import org.janusgraph.diskstorage.keycolumnvalue.SliceQuery; +import org.janusgraph.diskstorage.keycolumnvalue.StoreTransaction; +import org.janusgraph.diskstorage.util.StaticArrayBuffer; +import org.janusgraph.diskstorage.util.StaticArrayEntry.GetColVal; +import org.janusgraph.diskstorage.util.StaticArrayEntryList; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Metadata; +import com.datastax.driver.core.PreparedStatement; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.exceptions.QueryValidationException; +import com.datastax.driver.core.exceptions.UnsupportedFeatureException; +import com.datastax.driver.core.schemabuilder.TableOptions.CompactionOptions; +import com.datastax.driver.core.schemabuilder.TableOptions.CompressionOptions; +import com.google.common.collect.Lists; + +import javaslang.Lazy; +import javaslang.Tuple; +import javaslang.Tuple3; +import javaslang.collection.Array; +import javaslang.collection.Iterator; +import javaslang.concurrent.Future; +import javaslang.control.Try; +import javaslang.control.Try.FatalException; + +/** + * An implementation of {@link KeyColumnValueStore} which stores the data in a CQL connected backend. + */ +public class CQLKeyColumnValueStore implements KeyColumnValueStore { + + private static final String TTL_FUNCTION_NAME = "ttl"; + private static final String WRITETIME_FUNCTION_NAME = "writetime"; + + static final String KEY_COLUMN_NAME = "key"; + static final String COLUMN_COLUMN_NAME = "column1"; + static final String VALUE_COLUMN_NAME = "value"; + static final String WRITETIME_COLUMN_NAME = "writetime"; + static final String TTL_COLUMN_NAME = "ttl"; + + private static final String KEY_BINDING = "key"; + private static final String COLUMN_BINDING = "column1"; + private static final String VALUE_BINDING = "value"; + private static final String TIMESTAMP_BINDING = "timestamp"; + private static final String TTL_BINDING = "ttl"; + private static final String SLICE_START_BINDING = "sliceStart"; + private static final String SLICE_END_BINDING = "sliceEnd"; + private static final String KEY_START_BINDING = "keyStart"; + private static final String KEY_END_BINDING = "keyEnd"; + private static final String LIMIT_BINDING = "maxRows"; + + static final Function EXCEPTION_MAPPER = cause -> Match(cause).of( + Case(instanceOf(QueryValidationException.class), qve -> new PermanentBackendException(qve)), + Case(instanceOf(UnsupportedFeatureException.class), ufe -> new PermanentBackendException(ufe)), + Case($(), t -> new TemporaryBackendException(t))); + + private final CQLStoreManager storeManager; + private final ExecutorService executorService; + private final Session session; + private final String tableName; + private final CQLColValGetter getter; + private final Runnable closer; + + private final PreparedStatement getSlice; + private final PreparedStatement getKeysAll; + private final PreparedStatement getKeysRanged; + private final PreparedStatement deleteColumn; + private final PreparedStatement insertColumn; + private final PreparedStatement insertColumnWithTTL; + + /** + * Creates an instance of the {@link KeyColumnValueStore} that stores the data in a CQL backed table. + * + * @param storeManager the {@link CQLStoreManager} that maintains the list of {@link CQLKeyColumnValueStore}s + * @param tableName the name of the database table for storing the key/column/values + * @param configuration data used in creating this store + * @param closer callback used to clean up references to this store in the store manager + */ + public CQLKeyColumnValueStore(final CQLStoreManager storeManager, final String tableName, final Configuration configuration, final Runnable closer) { + this.storeManager = storeManager; + this.executorService = this.storeManager.getExecutorService(); + this.tableName = tableName; + this.closer = closer; + this.session = this.storeManager.getSession(); + this.getter = new CQLColValGetter(storeManager.getMetaDataSchema(this.tableName)); + + initializeTable(this.session, this.storeManager.getKeyspaceName(), tableName, configuration); + + // @formatter:off + this.getSlice = this.session.prepare(select() + .column(COLUMN_COLUMN_NAME) + .column(VALUE_COLUMN_NAME) + .fcall(WRITETIME_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(WRITETIME_COLUMN_NAME) + .fcall(TTL_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(TTL_COLUMN_NAME) + .from(this.storeManager.getKeyspaceName(), this.tableName) + .where(eq(KEY_COLUMN_NAME, bindMarker(KEY_BINDING))) + .and(gte(COLUMN_COLUMN_NAME, bindMarker(SLICE_START_BINDING))) + .and(lt(COLUMN_COLUMN_NAME, bindMarker(SLICE_END_BINDING))) + .limit(bindMarker(LIMIT_BINDING))); + + this.getKeysRanged = this.session.prepare(select() + .column(KEY_COLUMN_NAME) + .column(COLUMN_COLUMN_NAME) + .column(VALUE_COLUMN_NAME) + .fcall(WRITETIME_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(WRITETIME_COLUMN_NAME) + .fcall(TTL_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(TTL_COLUMN_NAME) + .from(this.storeManager.getKeyspaceName(), this.tableName) + .allowFiltering() + .where(gte(token(KEY_COLUMN_NAME), bindMarker(KEY_START_BINDING))) + .and(lt(token(KEY_COLUMN_NAME), bindMarker(KEY_END_BINDING))) + .and(gte(COLUMN_COLUMN_NAME, bindMarker(SLICE_START_BINDING))) + .and(lte(COLUMN_COLUMN_NAME, bindMarker(SLICE_END_BINDING)))); + + this.getKeysAll = this.session.prepare(select() + .column(KEY_COLUMN_NAME) + .column(COLUMN_COLUMN_NAME) + .column(VALUE_COLUMN_NAME) + .fcall(WRITETIME_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(WRITETIME_COLUMN_NAME) + .fcall(TTL_FUNCTION_NAME, column(VALUE_COLUMN_NAME)).as(TTL_COLUMN_NAME) + .from(this.storeManager.getKeyspaceName(), this.tableName) + .allowFiltering() + .where(gte(COLUMN_COLUMN_NAME, bindMarker(SLICE_START_BINDING))) + .and(lte(COLUMN_COLUMN_NAME, bindMarker(SLICE_END_BINDING)))); + + this.deleteColumn = this.session.prepare(delete() + .from(this.storeManager.getKeyspaceName(), this.tableName) + .where(eq(KEY_COLUMN_NAME, bindMarker(KEY_BINDING))) + .and(eq(COLUMN_COLUMN_NAME, bindMarker(COLUMN_BINDING))) + .using(timestamp(bindMarker(TIMESTAMP_BINDING)))); + + this.insertColumn = this.session.prepare(insertInto(this.storeManager.getKeyspaceName(), this.tableName) + .value(KEY_COLUMN_NAME, bindMarker(KEY_BINDING)) + .value(COLUMN_COLUMN_NAME, bindMarker(COLUMN_BINDING)) + .value(VALUE_COLUMN_NAME, bindMarker(VALUE_BINDING)) + .using(timestamp(bindMarker(TIMESTAMP_BINDING)))); + + this.insertColumnWithTTL = this.session.prepare(insertInto(this.storeManager.getKeyspaceName(), this.tableName) + .value(KEY_COLUMN_NAME, bindMarker(KEY_BINDING)) + .value(COLUMN_COLUMN_NAME, bindMarker(COLUMN_BINDING)) + .value(VALUE_COLUMN_NAME, bindMarker(VALUE_BINDING)) + .using(timestamp(bindMarker(TIMESTAMP_BINDING))) + .and(ttl(bindMarker(TTL_BINDING)))); + // @formatter:on + } + + private static void initializeTable(final Session session, final String keyspaceName, final String tableName, final Configuration configuration) { + session.execute(createTable(keyspaceName, tableName) + .ifNotExists() + .addPartitionKey(KEY_COLUMN_NAME, DataType.blob()) + .addClusteringColumn(COLUMN_COLUMN_NAME, DataType.blob()) + .addColumn(VALUE_COLUMN_NAME, DataType.blob()) + .withOptions() + .compressionOptions(compressionOptions(configuration)) + .compactionOptions(compactionOptions(configuration)) + .compactStorage()); + } + + private static CompressionOptions compressionOptions(final Configuration configuration) { + if (!configuration.get(CF_COMPRESSION)) { + // No compression + return noCompression(); + } + + return Match(configuration.get(CF_COMPRESSION_TYPE)).of( + Case("LZ4Compressor", lz4()), + Case("SnappyCompressor", snappy()), + Case("DeflateCompressor", deflate())) + .withChunkLengthInKb(configuration.get(CF_COMPRESSION_BLOCK_SIZE)); + } + + private static CompactionOptions compactionOptions(final Configuration configuration) { + if (!configuration.has(COMPACTION_STRATEGY)) { + return null; + } + + final CompactionOptions compactionOptions = Match(configuration.get(COMPACTION_STRATEGY)) + .of( + Case("SizeTieredCompactionStrategy", sizedTieredStategy()), + Case("DateTieredCompactionStrategy", dateTieredStrategy()), + Case("LeveledCompactionStrategy", leveledStrategy())); + Array.of(configuration.get(COMPACTION_OPTIONS)) + .grouped(2) + .forEach(keyValue -> compactionOptions.freeformOption(keyValue.get(0), keyValue.get(1))); + return compactionOptions; + } + + @Override + public void close() throws BackendException { + this.closer.run(); + } + + @Override + public String getName() { + return this.tableName; + } + + @Override + public EntryList getSlice(final KeySliceQuery query, final StoreTransaction txh) throws BackendException { + final Future result = Future.fromJavaFuture( + this.executorService, + this.session.executeAsync(this.getSlice.bind() + .setBytes(KEY_BINDING, query.getKey().asByteBuffer()) + .setBytes(SLICE_START_BINDING, query.getSliceStart().asByteBuffer()) + .setBytes(SLICE_END_BINDING, query.getSliceEnd().asByteBuffer()) + .setInt(LIMIT_BINDING, query.getLimit()) + .setConsistencyLevel(getTransaction(txh).getReadConsistencyLevel()))) + .map(resultSet -> fromResultSet(resultSet, this.getter)); + awaitInterruptibly(result); + return result.getValue().get().getOrElseThrow(EXCEPTION_MAPPER); + } + + @Override + public Map getSlice(final List keys, final SliceQuery query, final StoreTransaction txh) throws BackendException { + throw new UnsupportedOperationException("The CQL backend does not support multi-key queries"); + } + + /** + * Javaslang Future.await will throw InterruptedException wrapped in a FatalException. If the Thread was in Object.wait, the interrupted + * flag will be cleared as a side effect and needs to be reset. This method checks that the underlying cause of the FatalException is + * InterruptedException and resets the interrupted flag. + * + * @param result the future to wait on + * @throws PermanentBackendException if the thread was interrupted while waiting for the future result + */ + private void awaitInterruptibly(final Future result) throws PermanentBackendException { + try { + result.await(); + } catch (FatalException e) { + Throwable cause = e.getCause(); + if (cause instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new PermanentBackendException(cause); + } + } + + private static EntryList fromResultSet(final ResultSet resultSet, final GetColVal, StaticBuffer> getter) { + final Lazy> lazyList = Lazy.of(() -> Lists.newArrayList(resultSet)); + // Use the Iterable overload of ofByteBuffer as it's able to allocate + // the byte array up front. + // To ensure that the Iterator instance is recreated, it is created + // within the closure otherwise + // the same iterator would be reused and would be exhausted. + return StaticArrayEntryList.ofStaticBuffer(() -> Iterator.ofAll(lazyList.get()) + .> map(row -> Tuple.of( + StaticArrayBuffer.of(row.getBytes(COLUMN_COLUMN_NAME)), + StaticArrayBuffer.of(row.getBytes(VALUE_COLUMN_NAME)), + row)), + getter); + } + + /* + * Used from CQLStoreManager + */ + Statement deleteColumn(final StaticBuffer key, final StaticBuffer column, final long timestamp) { + return this.deleteColumn.bind() + .setBytes(KEY_BINDING, key.asByteBuffer()) + .setBytes(COLUMN_BINDING, column.asByteBuffer()) + .setLong(TIMESTAMP_BINDING, timestamp); + } + + /* + * Used from CQLStoreManager + */ + Statement insertColumn(final StaticBuffer key, final Entry entry, final long timestamp) { + final Integer ttl = (Integer) entry.getMetaData().get(EntryMetaData.TTL); + if (ttl != null) { + return this.insertColumnWithTTL.bind() + .setBytes(KEY_BINDING, key.asByteBuffer()) + .setBytes(COLUMN_BINDING, entry.getColumn().asByteBuffer()) + .setBytes(VALUE_BINDING, entry.getValue().asByteBuffer()) + .setLong(TIMESTAMP_BINDING, timestamp) + .setInt(TTL_BINDING, ttl); + } + return this.insertColumn.bind() + .setBytes(KEY_BINDING, key.asByteBuffer()) + .setBytes(COLUMN_BINDING, entry.getColumn().asByteBuffer()) + .setBytes(VALUE_BINDING, entry.getValue().asByteBuffer()) + .setLong(TIMESTAMP_BINDING, timestamp); + } + + @Override + public void mutate(final StaticBuffer key, final List additions, final List deletions, final StoreTransaction txh) throws BackendException { + this.storeManager.mutateMany(Collections.singletonMap(this.tableName, Collections.singletonMap(key, new KCVMutation(additions, deletions))), txh); + } + + @Override + public void acquireLock(final StaticBuffer key, final StaticBuffer column, final StaticBuffer expectedValue, final StoreTransaction txh) throws BackendException { + throw new UnsupportedOperationException(); + } + + @Override + public KeyIterator getKeys(final KeyRangeQuery query, final StoreTransaction txh) throws BackendException { + if (!this.storeManager.getFeatures().hasOrderedScan()) { + throw new PermanentBackendException("This operation is only allowed when the byteorderedpartitioner is used."); + } + + final Metadata metadata = this.session.getCluster().getMetadata(); + return Try.of(() -> new CQLResultSetKeyIterator( + query, + this.getter, + this.session.execute(this.getKeysRanged.bind() + .setToken(KEY_START_BINDING, metadata.newToken(query.getKeyStart().asByteBuffer())) + .setToken(KEY_END_BINDING, metadata.newToken(query.getKeyEnd().asByteBuffer())) + .setBytes(SLICE_START_BINDING, query.getSliceStart().asByteBuffer()) + .setBytes(SLICE_END_BINDING, query.getSliceEnd().asByteBuffer()) + .setFetchSize(this.storeManager.getPageSize()) + .setConsistencyLevel(getTransaction(txh).getReadConsistencyLevel())))) + .getOrElseThrow(EXCEPTION_MAPPER); + } + + @Override + public KeyIterator getKeys(final SliceQuery query, final StoreTransaction txh) throws BackendException { + if (this.storeManager.getFeatures().hasOrderedScan()) { + throw new PermanentBackendException("This operation is only allowed when a random partitioner (md5 or murmur3) is used."); + } + + return Try.of(() -> new CQLResultSetKeyIterator( + query, + this.getter, + this.session.execute(this.getKeysAll.bind() + .setBytes(SLICE_START_BINDING, query.getSliceStart().asByteBuffer()) + .setBytes(SLICE_END_BINDING, query.getSliceEnd().asByteBuffer()) + .setFetchSize(this.storeManager.getPageSize()) + .setConsistencyLevel(getTransaction(txh).getReadConsistencyLevel())))) + .getOrElseThrow(EXCEPTION_MAPPER); + } +} diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIterator.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIterator.java new file mode 100644 index 0000000000..ae0e8931fd --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIterator.java @@ -0,0 +1,118 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import java.io.IOException; + +import org.janusgraph.diskstorage.Entry; +import org.janusgraph.diskstorage.StaticBuffer; +import org.janusgraph.diskstorage.keycolumnvalue.KeyIterator; +import org.janusgraph.diskstorage.keycolumnvalue.SliceQuery; +import org.janusgraph.diskstorage.util.RecordIterator; +import org.janusgraph.diskstorage.util.StaticArrayBuffer; +import org.janusgraph.diskstorage.util.StaticArrayEntry; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; +import com.google.common.collect.AbstractIterator; + +import javaslang.Tuple; +import javaslang.Tuple3; +import javaslang.collection.Iterator; + +/** + * {@link SliceQuery} iterator that handles CQL result sets that may have more + * data returned in each column than the {@link SliceQuery} has configured as + * it's limit. I.e. the iterator only returns the number of entries for each Key + * to the number of Columns specified in the {@link SliceQuery}s limit. + */ +class CQLResultSetKeyIterator extends AbstractIterator implements KeyIterator { + + private final SliceQuery sliceQuery; + private final CQLColValGetter getter; + private final Iterator iterator; + + private Row currentRow = null; + private StaticBuffer currentKey = null; + private StaticBuffer lastKey = null; + + CQLResultSetKeyIterator(final SliceQuery sliceQuery, final CQLColValGetter getter, final ResultSet resultSet) { + this.sliceQuery = sliceQuery; + this.getter = getter; + this.iterator = Iterator.ofAll(resultSet.iterator()) + .peek(row -> { + this.currentRow = row; + this.currentKey = StaticArrayBuffer.of(row.getBytes(CQLKeyColumnValueStore.KEY_COLUMN_NAME)); + }); + } + + @Override + protected StaticBuffer computeNext() { + if (this.currentKey != null && !this.currentKey.equals(this.lastKey)) { + this.lastKey = this.currentKey; + return this.lastKey; + } + + while (this.iterator.hasNext()) { + this.iterator.next(); + if (!this.currentKey.equals(this.lastKey)) { + this.lastKey = this.currentKey; + return this.lastKey; + } + } + return endOfData(); + } + + @Override + public RecordIterator getEntries() { + return new EntryRecordIterator(this.sliceQuery, this.getter, Iterator.of(this.currentRow).concat(this.iterator), this.currentKey); + } + + @Override + public void close() throws IOException { + // NOP + } + + static class EntryRecordIterator extends AbstractIterator implements RecordIterator { + + private final CQLColValGetter getter; + private final Iterator> iterator; + + EntryRecordIterator(final SliceQuery sliceQuery, final CQLColValGetter getter, final Iterator iterator, final StaticBuffer key) { + this.getter = getter; + final StaticBuffer sliceEnd = sliceQuery.getSliceEnd(); + this.iterator = iterator + .> map(row -> Tuple.of( + StaticArrayBuffer.of(row.getBytes(CQLKeyColumnValueStore.COLUMN_COLUMN_NAME)), + StaticArrayBuffer.of(row.getBytes(CQLKeyColumnValueStore.VALUE_COLUMN_NAME)), + row)) + .takeWhile(tuple -> key.equals(StaticArrayBuffer.of(tuple._3.getBytes(CQLKeyColumnValueStore.KEY_COLUMN_NAME))) && !sliceEnd.equals(tuple._1)) + .take(sliceQuery.getLimit()); + } + + @Override + protected Entry computeNext() { + if (this.iterator.hasNext()) { + return StaticArrayEntry.ofStaticBuffer(this.iterator.next(), this.getter); + } + return endOfData(); + } + + @Override + public void close() throws IOException { + // NOP + } + } +} diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLStoreManager.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLStoreManager.java new file mode 100644 index 0000000000..e6896e90cc --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLStoreManager.java @@ -0,0 +1,414 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.createKeyspace; +import static com.datastax.driver.core.schemabuilder.SchemaBuilder.dropKeyspace; +import static javaslang.API.$; +import static javaslang.API.Case; +import static javaslang.API.Match; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.ATOMIC_BATCH_MUTATE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.BATCH_STATEMENT_SIZE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.CLUSTER_NAME; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.KEYSPACE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.LOCAL_DATACENTER; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.PROTOCOL_VERSION; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.READ_CONSISTENCY; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.REPLICATION_FACTOR; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.REPLICATION_OPTIONS; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.REPLICATION_STRATEGY; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_ENABLED; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_TRUSTSTORE_LOCATION; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_TRUSTSTORE_PASSWORD; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.WRITE_CONSISTENCY; +import static org.janusgraph.diskstorage.cql.CQLKeyColumnValueStore.EXCEPTION_MAPPER; +import static org.janusgraph.diskstorage.cql.CQLTransaction.getTransaction; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.AUTH_PASSWORD; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.AUTH_USERNAME; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.METRICS_PREFIX; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.METRICS_SYSTEM_PREFIX_DEFAULT; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.buildGraphConfiguration; + +import java.io.FileInputStream; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.security.KeyManagementException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.cert.CertificateException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.BaseTransactionConfig; +import org.janusgraph.diskstorage.PermanentBackendException; +import org.janusgraph.diskstorage.StaticBuffer; +import org.janusgraph.diskstorage.StoreMetaData.Container; +import org.janusgraph.diskstorage.common.DistributedStoreManager; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.keycolumnvalue.KCVMutation; +import org.janusgraph.diskstorage.keycolumnvalue.KeyColumnValueStore; +import org.janusgraph.diskstorage.keycolumnvalue.KeyColumnValueStoreManager; +import org.janusgraph.diskstorage.keycolumnvalue.KeyRange; +import org.janusgraph.diskstorage.keycolumnvalue.StandardStoreFeatures; +import org.janusgraph.diskstorage.keycolumnvalue.StoreFeatures; +import org.janusgraph.diskstorage.keycolumnvalue.StoreTransaction; +import org.janusgraph.util.system.NetworkUtil; + +import com.datastax.driver.core.BatchStatement; +import com.datastax.driver.core.BatchStatement.Type; +import com.datastax.driver.core.Cluster; +import com.datastax.driver.core.Cluster.Builder; +import com.datastax.driver.core.JdkSSLOptions; +import com.datastax.driver.core.KeyspaceMetadata; +import com.datastax.driver.core.ProtocolVersion; +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.policies.DCAwareRoundRobinPolicy; +import com.datastax.driver.core.policies.TokenAwarePolicy; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import javaslang.Tuple; +import javaslang.collection.Array; +import javaslang.collection.HashMap; +import javaslang.collection.Iterator; +import javaslang.collection.Seq; +import javaslang.concurrent.Future; +import javaslang.control.Option; + +/** + * This class creates {@see CQLKeyColumnValueStore}s and handles Cassandra-backed allocation of vertex IDs for JanusGraph (when so + * configured). + */ +public class CQLStoreManager extends DistributedStoreManager implements KeyColumnValueStoreManager { + + static final String CONSISTENCY_LOCAL_QUORUM = "LOCAL_QUORUM"; + static final String CONSISTENCY_QUORUM = "QUORUM"; + + private static final int DEFAULT_PORT = 9042; + + private final String keyspace; + private final int batchSize; + private final boolean atomicBatch; + + private final ExecutorService executorService; + + private final Cluster cluster; + private final Session session; + private final StoreFeatures storeFeatures; + private final Map openStores; + private final Deployment deployment; + + /** + * Constructor for the {@link CQLStoreManager} given a JanusGraph {@link Configuration}. + */ + public CQLStoreManager(final Configuration configuration) throws BackendException { + super(configuration, DEFAULT_PORT); + this.keyspace = configuration.get(KEYSPACE); + this.batchSize = configuration.get(BATCH_STATEMENT_SIZE); + this.atomicBatch = configuration.get(ATOMIC_BATCH_MUTATE); + + this.executorService = new ThreadPoolExecutor(10, + 100, + 1, + TimeUnit.MINUTES, + new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder() + .setDaemon(true) + .setNameFormat("CQLStoreManager[%02d]") + .build()); + + this.cluster = initializeCluster(); + this.session = initializeSession(this.keyspace); + + final Configuration global = buildGraphConfiguration() + .set(READ_CONSISTENCY, CONSISTENCY_QUORUM) + .set(WRITE_CONSISTENCY, CONSISTENCY_QUORUM) + .set(METRICS_PREFIX, METRICS_SYSTEM_PREFIX_DEFAULT); + + final Configuration local = buildGraphConfiguration() + .set(READ_CONSISTENCY, CONSISTENCY_LOCAL_QUORUM) + .set(WRITE_CONSISTENCY, CONSISTENCY_LOCAL_QUORUM) + .set(METRICS_PREFIX, METRICS_SYSTEM_PREFIX_DEFAULT); + + final StandardStoreFeatures.Builder fb = new StandardStoreFeatures.Builder(); + + fb.batchMutation(true).distributed(true); + fb.timestamps(true).cellTTL(true); + fb.keyConsistent(global, local); + fb.optimisticLocking(true); + fb.multiQuery(false); + + final String partitioner = this.cluster.getMetadata().getPartitioner(); + switch (partitioner.substring(partitioner.lastIndexOf('.') + 1)) { + case "RandomPartitioner": + case "Murmur3Partitioner": { + fb.keyOrdered(false).orderedScan(false).unorderedScan(true); + deployment = Deployment.REMOTE; + break; + } + case "ByteOrderedPartitioner": { + fb.keyOrdered(true).orderedScan(true).unorderedScan(false); + deployment = (hostnames.length == 1)// mark deployment as local only in case we have byte ordered partitioner and local + // connection + ? (NetworkUtil.isLocalConnection(hostnames[0])) ? Deployment.LOCAL : Deployment.REMOTE + : Deployment.REMOTE; + break; + } + default: { + throw new IllegalArgumentException("Unrecognized partitioner: " + partitioner); + } + } + this.storeFeatures = fb.build(); + this.openStores = new ConcurrentHashMap<>(); + } + + private Cluster initializeCluster() throws PermanentBackendException { + final Configuration configuration = getStorageConfig(); + + final List contactPoints; + try { + contactPoints = Array.of(this.hostnames) + .map(hostName -> hostName.split(":")) + .map(array -> Tuple.of(array[0], array.length == 2 ? Integer.parseInt(array[1]) : this.port)) + .map(tuple -> new InetSocketAddress(tuple._1, tuple._2)) + .toJavaList(); + } catch (SecurityException | ArrayIndexOutOfBoundsException | NumberFormatException e) { + throw new PermanentBackendException("Error initialising cluster contact points", e); + } + + final Builder builder = Cluster.builder() + .addContactPointsWithPorts(contactPoints) + .withClusterName(configuration.get(CLUSTER_NAME)); + + if (configuration.get(PROTOCOL_VERSION) == 0) { + builder.withProtocolVersion(ProtocolVersion.NEWEST_SUPPORTED); + } else { + builder.withProtocolVersion(ProtocolVersion.fromInt(configuration.get(PROTOCOL_VERSION))); + } + if (configuration.has(AUTH_USERNAME) && configuration.has(AUTH_PASSWORD)) { + builder.withCredentials(configuration.get(AUTH_USERNAME), configuration.get(AUTH_PASSWORD)); + } + + if (configuration.has(LOCAL_DATACENTER)) { + builder.withLoadBalancingPolicy(new TokenAwarePolicy(DCAwareRoundRobinPolicy.builder() + .withLocalDc(configuration.get(LOCAL_DATACENTER)) + .build())); + } + + if (configuration.get(SSL_ENABLED)) { + try { + final TrustManager[] trustManagers; + try (final FileInputStream keyStoreStream = new FileInputStream(configuration.get(SSL_TRUSTSTORE_LOCATION))) { + final KeyStore keystore = KeyStore.getInstance("jks"); + keystore.load(keyStoreStream, configuration.get(SSL_TRUSTSTORE_PASSWORD).toCharArray()); + final TrustManagerFactory trustManagerFactory = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + trustManagerFactory.init(keystore); + trustManagers = trustManagerFactory.getTrustManagers(); + } + + final SSLContext sslContext = SSLContext.getInstance("TLS"); + sslContext.init(null, trustManagers, null); + + final JdkSSLOptions sslOptions = JdkSSLOptions.builder() + .withSSLContext(sslContext) + .build(); + builder.withSSL(sslOptions); + + } catch (NoSuchAlgorithmException | CertificateException | IOException | KeyStoreException | KeyManagementException e) { + throw new PermanentBackendException("Error initialising SSL connection properties", e); + } + } + + return builder.build(); + } + + private Session initializeSession(final String keyspaceName) { + final Configuration configuration = getStorageConfig(); + final Map replication = Match(configuration.get(REPLICATION_STRATEGY)).of( + Case($("SimpleStrategy"), strategy -> HashMap. of("class", strategy, "replication_factor", configuration.get(REPLICATION_FACTOR))), + Case($("NetworkTopologyStrategy"), + strategy -> HashMap. of("class", strategy) + .merge(Array.of(configuration.get(REPLICATION_OPTIONS)) + .grouped(2) + .toMap(array -> Tuple.of(array.get(0), Integer.parseInt(array.get(1))))))) + .toJavaMap(); + + final Session s = this.cluster.connect(); + s.execute(createKeyspace(keyspaceName) + .ifNotExists() + .with() + .replication(replication)); + return s; + } + + ExecutorService getExecutorService() { + return this.executorService; + } + + Session getSession() { + return this.session; + } + + String getKeyspaceName() { + return this.keyspace; + } + + Map getCompressionOptions(final String name) throws BackendException { + final KeyspaceMetadata keyspaceMetadata = Option.of(this.cluster.getMetadata().getKeyspace(this.keyspace)) + .getOrElseThrow(() -> new PermanentBackendException(String.format("Unknown keyspace '%s'", this.keyspace))); + return Option.of(keyspaceMetadata.getTable(name)) + .map(tableMetadata -> tableMetadata.getOptions().getCompression()) + .getOrElseThrow(() -> new PermanentBackendException(String.format("Unknown table '%s'", name))); + } + + @Override + public void close() throws BackendException { + try { + this.session.close(); + } finally { + try { + this.cluster.close(); + } finally { + this.executorService.shutdownNow(); + } + } + } + + @Override + public String getName() { + return String.format("%s.%s", getClass().getSimpleName(), this.keyspace); + } + + @Override + public Deployment getDeployment() { + return this.deployment; + } + + @Override + public StoreFeatures getFeatures() { + return this.storeFeatures; + } + + @Override + public KeyColumnValueStore openDatabase(final String name, final Container metaData) throws BackendException { + return this.openStores.computeIfAbsent(name, n -> new CQLKeyColumnValueStore(this, n, getStorageConfig(), () -> this.openStores.remove(n))); + } + + @Override + public StoreTransaction beginTransaction(final BaseTransactionConfig config) throws BackendException { + return new CQLTransaction(config); + } + + @Override + public void clearStorage() throws BackendException { + this.session.execute(dropKeyspace(this.keyspace)); + } + + @Override + public List getLocalKeyPartition() throws BackendException { + throw new UnsupportedOperationException(); + } + + @Override + public void mutateMany(final Map> mutations, final StoreTransaction txh) throws BackendException { + if (this.atomicBatch) { + mutateManyLogged(mutations, txh); + } else { + mutateManyUnlogged(mutations, txh); + } + } + + // Use a single logged batch + private void mutateManyLogged(final Map> mutations, final StoreTransaction txh) throws BackendException { + final MaskedTimestamp commitTime = new MaskedTimestamp(txh); + + final BatchStatement batchStatement = new BatchStatement(Type.LOGGED); + batchStatement.setConsistencyLevel(getTransaction(txh).getWriteConsistencyLevel()); + + batchStatement.addAll(Iterator.ofAll(mutations.entrySet()).flatMap(tableNameAndMutations -> { + final String tableName = tableNameAndMutations.getKey(); + final Map tableMutations = tableNameAndMutations.getValue(); + + final CQLKeyColumnValueStore columnValueStore = Option.of(this.openStores.get(tableName)) + .getOrElseThrow(() -> new IllegalStateException("Store cannot be found: " + tableName)); + return Iterator.ofAll(tableMutations.entrySet()).flatMap(keyAndMutations -> { + final StaticBuffer key = keyAndMutations.getKey(); + final KCVMutation keyMutations = keyAndMutations.getValue(); + + final Iterator deletions = Iterator.of(commitTime.getDeletionTime(this.times)) + .flatMap(deleteTime -> Iterator.ofAll(keyMutations.getDeletions()).map(deletion -> columnValueStore.deleteColumn(key, deletion, deleteTime))); + final Iterator additions = Iterator.of(commitTime.getAdditionTime(this.times)) + .flatMap(addTime -> Iterator.ofAll(keyMutations.getAdditions()).map(addition -> columnValueStore.insertColumn(key, addition, addTime))); + + return Iterator.concat(deletions, additions); + }); + })); + final Future result = Future.fromJavaFuture(this.executorService, this.session.executeAsync(batchStatement)); + + result.await(); + if (result.isFailure()) { + throw EXCEPTION_MAPPER.apply(result.getCause().get()); + } + sleepAfterWrite(txh, commitTime); + } + + // Create an async unlogged batch per partition key + private void mutateManyUnlogged(final Map> mutations, final StoreTransaction txh) throws BackendException { + final MaskedTimestamp commitTime = new MaskedTimestamp(txh); + + final Future> result = Future.sequence(this.executorService, Iterator.ofAll(mutations.entrySet()).flatMap(tableNameAndMutations -> { + final String tableName = tableNameAndMutations.getKey(); + final Map tableMutations = tableNameAndMutations.getValue(); + + final CQLKeyColumnValueStore columnValueStore = Option.of(this.openStores.get(tableName)) + .getOrElseThrow(() -> new IllegalStateException("Store cannot be found: " + tableName)); + return Iterator.ofAll(tableMutations.entrySet()).flatMap(keyAndMutations -> { + final StaticBuffer key = keyAndMutations.getKey(); + final KCVMutation keyMutations = keyAndMutations.getValue(); + + final Iterator deletions = Iterator.of(commitTime.getDeletionTime(this.times)) + .flatMap(deleteTime -> Iterator.ofAll(keyMutations.getDeletions()).map(deletion -> columnValueStore.deleteColumn(key, deletion, deleteTime))); + final Iterator additions = Iterator.of(commitTime.getAdditionTime(this.times)) + .flatMap(addTime -> Iterator.ofAll(keyMutations.getAdditions()).map(addition -> columnValueStore.insertColumn(key, addition, addTime))); + + return Iterator.concat(deletions, additions) + .grouped(this.batchSize) + .map(group -> Future.fromJavaFuture(this.executorService, + this.session.executeAsync( + new BatchStatement(Type.UNLOGGED) + .addAll(group) + .setConsistencyLevel(getTransaction(txh).getWriteConsistencyLevel())))); + }); + })); + + result.await(); + if (result.isFailure()) { + throw EXCEPTION_MAPPER.apply(result.getCause().get()); + } + sleepAfterWrite(txh, commitTime); + } +} diff --git a/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLTransaction.java b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLTransaction.java new file mode 100644 index 0000000000..a2410d7e09 --- /dev/null +++ b/janusgraph-cql/src/main/java/org/janusgraph/diskstorage/cql/CQLTransaction.java @@ -0,0 +1,66 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.*; + +import org.janusgraph.diskstorage.BaseTransactionConfig; +import org.janusgraph.diskstorage.common.AbstractStoreTransaction; +import org.janusgraph.diskstorage.keycolumnvalue.StoreTransaction; + +import com.datastax.driver.core.ConsistencyLevel; +import com.google.common.base.Preconditions; + +/** + * This class manages the translation of read and write consistency configuration values to CQL API {@link ConsistencyLevel} types. + */ +public class CQLTransaction extends AbstractStoreTransaction { + + private final ConsistencyLevel readConsistencyLevel; + private final ConsistencyLevel writeConsistencyLevel; + + public CQLTransaction(final BaseTransactionConfig config) { + super(config); + this.readConsistencyLevel = ConsistencyLevel.valueOf(getConfiguration().getCustomOption(READ_CONSISTENCY)); + this.writeConsistencyLevel = ConsistencyLevel.valueOf(getConfiguration().getCustomOption(WRITE_CONSISTENCY)); + } + + ConsistencyLevel getReadConsistencyLevel() { + return this.readConsistencyLevel; + } + + ConsistencyLevel getWriteConsistencyLevel() { + return this.writeConsistencyLevel; + } + + static CQLTransaction getTransaction(final StoreTransaction storeTransaction) { + Preconditions.checkArgument(storeTransaction != null); + Preconditions.checkArgument(storeTransaction instanceof CQLTransaction, "Unexpected transaction type %s", storeTransaction.getClass().getName()); + return (CQLTransaction) storeTransaction; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder(64); + sb.append("CQLTransaction@"); + sb.append(Integer.toHexString(hashCode())); + sb.append("[read="); + sb.append(this.readConsistencyLevel); + sb.append(",write="); + sb.append(this.writeConsistencyLevel); + sb.append("]"); + return sb.toString(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphComputerProvider.java b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphComputerProvider.java new file mode 100644 index 0000000000..12e1790863 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphComputerProvider.java @@ -0,0 +1,34 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.cql; + +import org.apache.tinkerpop.gremlin.GraphProvider; +import org.janusgraph.blueprints.AbstractJanusGraphComputerProvider; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.olap.computer.FulgoraGraphComputer; + +@GraphProvider.Descriptor(computer = FulgoraGraphComputer.class) +public class CQLGraphComputerProvider extends AbstractJanusGraphComputerProvider { + + @Override + public ModifiableConfiguration getJanusGraphConfiguration(String graphName, Class test, String testMethodName) { + CassandraStorageSetup.startCleanEmbedded(); + ModifiableConfiguration config = super.getJanusGraphConfiguration(graphName, test, testMethodName); + config.setAll(CassandraStorageSetup.getCQLConfiguration(graphName).getAll()); + return config; + } + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphProvider.java b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphProvider.java new file mode 100644 index 0000000000..9189f1ad9b --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/CQLGraphProvider.java @@ -0,0 +1,29 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.cql; + +import org.janusgraph.blueprints.AbstractJanusGraphProvider; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; + +public class CQLGraphProvider extends AbstractJanusGraphProvider { + + @Override + public ModifiableConfiguration getJanusGraphConfiguration(String graphName, Class test, String testMethodName) { + CassandraStorageSetup.startCleanEmbedded(); + return CassandraStorageSetup.getCQLConfiguration(graphName); + } + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLComputerTest.java b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLComputerTest.java new file mode 100644 index 0000000000..bfbbb9d0fa --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLComputerTest.java @@ -0,0 +1,29 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.cql.process; + +import org.apache.tinkerpop.gremlin.GraphProviderClass; +import org.apache.tinkerpop.gremlin.process.ProcessComputerSuite; +import org.janusgraph.blueprints.cql.CQLGraphComputerProvider; +import org.janusgraph.core.JanusGraph; +import org.junit.runner.RunWith; + +@RunWith(ProcessComputerSuite.class) +@GraphProviderClass(provider = CQLGraphComputerProvider.class, graph = JanusGraph.class) +public class CQLComputerTest { + +// TP3 ignores @BeforeClass + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLProcessTest.java b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLProcessTest.java new file mode 100644 index 0000000000..dea6673934 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/process/CQLProcessTest.java @@ -0,0 +1,29 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.cql.process; + +import org.apache.tinkerpop.gremlin.GraphProviderClass; +import org.apache.tinkerpop.gremlin.process.ProcessStandardSuite; +import org.janusgraph.blueprints.cql.CQLGraphProvider; +import org.janusgraph.core.JanusGraph; +import org.junit.runner.RunWith; + +@RunWith(ProcessStandardSuite.class) +@GraphProviderClass(provider = CQLGraphProvider.class, graph = JanusGraph.class) +public class CQLProcessTest { + +// TP3 ignores @BeforeClass + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/structure/CQLStructureTest.java b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/structure/CQLStructureTest.java new file mode 100644 index 0000000000..72105a4e6b --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/blueprints/cql/structure/CQLStructureTest.java @@ -0,0 +1,29 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.blueprints.cql.structure; + +import org.apache.tinkerpop.gremlin.GraphProviderClass; +import org.apache.tinkerpop.gremlin.structure.StructureStandardSuite; +import org.janusgraph.blueprints.cql.CQLGraphProvider; +import org.janusgraph.core.JanusGraph; +import org.junit.runner.RunWith; + +@RunWith(StructureStandardSuite.class) +@GraphProviderClass(provider = CQLGraphProvider.class, graph = JanusGraph.class) +public class CQLStructureTest { + +// TP3 ignores @BeforeClass + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLDistributedStoreManagerTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLDistributedStoreManagerTest.java new file mode 100644 index 0000000000..3705b8e2d1 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLDistributedStoreManagerTest.java @@ -0,0 +1,41 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.DistributedStoreManagerTest; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; + +public class CQLDistributedStoreManagerTest extends DistributedStoreManagerTest { + + @BeforeClass + public static void startCassandra() { + CassandraStorageSetup.startCleanEmbedded(); + } + + @Before + public void setUp() throws BackendException { + manager = new CQLStoreManager(CassandraStorageSetup.getCQLConfiguration(this.getClass().getSimpleName())); + store = manager.openDatabase("distributedcf"); + } + + @After + public void tearDown() throws BackendException { + if (null != manager) + manager.close(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLIDAuthorityTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLIDAuthorityTest.java new file mode 100644 index 0000000000..b3cc8da5a7 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLIDAuthorityTest.java @@ -0,0 +1,49 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.IDAuthorityTest; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.junit.BeforeClass; + +public class CQLIDAuthorityTest extends IDAuthorityTest { + + public CQLIDAuthorityTest(final WriteConfiguration baseConfig) { + super(baseConfig); + } + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(); + } + + private ModifiableConfiguration getBaseStorageConfiguration() { + return getCQLConfiguration(getClass().getSimpleName()); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager() throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLockStoreTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLockStoreTest.java new file mode 100644 index 0000000000..46ea8b16e7 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLockStoreTest.java @@ -0,0 +1,44 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.LockKeyColumnValueStoreTest; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.junit.BeforeClass; + +public class CQLLockStoreTest extends LockKeyColumnValueStoreTest { + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(); + } + + private ModifiableConfiguration getBaseStorageConfiguration() { + return getCQLConfiguration(getClass().getSimpleName()); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager(final int idx, final Configuration configuration) throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLogTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLogTest.java new file mode 100644 index 0000000000..62ba66c73f --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLLogTest.java @@ -0,0 +1,47 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.diskstorage.log.KCVSLogTest; +import org.janusgraph.testcategory.SerialTests; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +@Category(SerialTests.class) +public class CQLLogTest extends KCVSLogTest { + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(); + } + + private ModifiableConfiguration getBaseStorageConfiguration() { + return getCQLConfiguration(getClass().getSimpleName()); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager() throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLMultiWriteStoreTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLMultiWriteStoreTest.java new file mode 100644 index 0000000000..c35aba87da --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLMultiWriteStoreTest.java @@ -0,0 +1,44 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.MultiWriteKeyColumnValueStoreTest; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.junit.BeforeClass; + +public class CQLMultiWriteStoreTest extends MultiWriteKeyColumnValueStoreTest { + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(); + } + + private ModifiableConfiguration getBaseStorageConfiguration() { + return getCQLConfiguration(getClass().getSimpleName()); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager() throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIteratorTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIteratorTest.java new file mode 100644 index 0000000000..b80a47de7c --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLResultSetKeyIteratorTest.java @@ -0,0 +1,235 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Random; + +import org.janusgraph.diskstorage.Entry; +import org.janusgraph.diskstorage.EntryMetaData; +import org.janusgraph.diskstorage.StaticBuffer; +import org.janusgraph.diskstorage.keycolumnvalue.SliceQuery; +import org.janusgraph.diskstorage.util.BufferUtil; +import org.janusgraph.diskstorage.util.RecordIterator; +import org.junit.Test; + +import com.datastax.driver.core.ResultSet; +import com.datastax.driver.core.Row; + +import javaslang.Function1; +import javaslang.Tuple; +import javaslang.Tuple2; +import javaslang.collection.Array; +import javaslang.collection.Iterator; +import javaslang.collection.Seq; + +public class CQLResultSetKeyIteratorTest { + + private static final SliceQuery ALL_COLUMNS = new SliceQuery(BufferUtil.zeroBuffer(1), BufferUtil.oneBuffer(128)); + + @Test + public void testIterator() throws IOException { + final Array rows = Array.rangeClosed(1, 100).map(idx -> { + final Row row = mock(Row.class); + when(row.getBytes("key")).thenReturn(ByteBuffer.wrap(Integer.toString(idx / 5).getBytes())); + when(row.getBytes("column1")).thenReturn(ByteBuffer.wrap(Integer.toString(idx % 5).getBytes())); + when(row.getBytes("value")).thenReturn(ByteBuffer.wrap(Integer.toString(idx).getBytes())); + return row; + }); + + final ResultSet resultSet = mock(ResultSet.class); + when(resultSet.iterator()).thenReturn(rows.iterator()); + + final CQLColValGetter getter = new CQLColValGetter(new EntryMetaData[0]); + try (final CQLResultSetKeyIterator resultSetKeyIterator = new CQLResultSetKeyIterator(ALL_COLUMNS, getter, resultSet)) { + int i = 0; + while (resultSetKeyIterator.hasNext()) { + final StaticBuffer next = resultSetKeyIterator.next(); + + final RecordIterator entries = resultSetKeyIterator.getEntries(); + while (entries.hasNext()) { + final Row row = rows.get(i++); + final Entry entry = entries.next(); + + assertEquals(row.getBytes("key"), next.asByteBuffer()); + assertEquals(row.getBytes("column1"), entry.getColumn().asByteBuffer()); + assertEquals(row.getBytes("value"), entry.getValue().asByteBuffer()); + } + } + } + } + + @Test + public void testEmpty() throws IOException { + final Array rows = Array.empty(); + + final ResultSet resultSet = mock(ResultSet.class); + when(resultSet.iterator()).thenReturn(rows.iterator()); + + final CQLColValGetter getter = new CQLColValGetter(new EntryMetaData[0]); + try (final CQLResultSetKeyIterator resultSetKeyIterator = new CQLResultSetKeyIterator(ALL_COLUMNS, getter, resultSet)) { + assertFalse(resultSetKeyIterator.hasNext()); + } + } + + @Test + public void testUneven() throws IOException { + final Random random = new Random(); + + final Function1 randomLong = idx -> { + final ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES).putLong(random.nextLong()); + buffer.flip(); + return buffer; + }; + + final Array>>> keysMap = Array.range(0, random.nextInt(100) + 100) + .map(randomLong) + .map(key -> Tuple.of(key, Array.rangeClosed(0, random.nextInt(100) + 1) + .map(idx -> Tuple.of(randomLong.apply(idx), randomLong.apply(idx))))); + + final Seq rows = keysMap.flatMap(tuple -> tuple._2.map(columnAndValue -> { + final Row row = mock(Row.class); + when(row.getBytes("key")).thenReturn(tuple._1); + when(row.getBytes("column1")).thenReturn(columnAndValue._1); + when(row.getBytes("value")).thenReturn(columnAndValue._2); + return row; + })); + + final ResultSet resultSet = mock(ResultSet.class); + when(resultSet.iterator()).thenReturn(rows.iterator()); + + final CQLColValGetter getter = new CQLColValGetter(new EntryMetaData[0]); + try (final CQLResultSetKeyIterator resultSetKeyIterator = new CQLResultSetKeyIterator(ALL_COLUMNS, getter, resultSet)) { + final Iterator>>> iterator = keysMap.iterator(); + while (resultSetKeyIterator.hasNext()) { + final StaticBuffer next = resultSetKeyIterator.next(); + try (final RecordIterator entries = resultSetKeyIterator.getEntries()) { + final Tuple2>> current = iterator.next(); + final ByteBuffer currentKey = current._1; + final Array> columnValues = current._2; + + final Iterator> columnIterator = columnValues.iterator(); + while (entries.hasNext()) { + final Entry entry = entries.next(); + final Tuple2 columnAndValue = columnIterator.next(); + + assertEquals(currentKey, next.asByteBuffer()); + assertEquals(columnAndValue._1, entry.getColumn().asByteBuffer()); + assertEquals(columnAndValue._2, entry.getValue().asByteBuffer()); + assertEquals(columnIterator.hasNext(), entries.hasNext()); + } + } + } + } + } + + @Test + public void testPartialIterateColumns() throws IOException { + final Random random = new Random(); + + final Function1 randomLong = idx -> { + final ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES).putLong(random.nextLong()); + buffer.flip(); + return buffer; + }; + + final Array>>> keysMap = Array.range(0, random.nextInt(100) + 100) + .map(randomLong) + .map(key -> Tuple.of(key, Array.rangeClosed(0, random.nextInt(100) + 1) + .map(idx -> Tuple.of(randomLong.apply(idx), randomLong.apply(idx))))); + + final Seq rows = keysMap.flatMap(tuple -> tuple._2.map(columnAndValue -> { + final Row row = mock(Row.class); + when(row.getBytes("key")).thenReturn(tuple._1); + when(row.getBytes("column1")).thenReturn(columnAndValue._1); + when(row.getBytes("value")).thenReturn(columnAndValue._2); + return row; + })); + + final ResultSet resultSet = mock(ResultSet.class); + when(resultSet.iterator()).thenReturn(rows.iterator()); + + final CQLColValGetter getter = new CQLColValGetter(new EntryMetaData[0]); + try (final CQLResultSetKeyIterator resultSetKeyIterator = new CQLResultSetKeyIterator(ALL_COLUMNS, getter, resultSet)) { + final Iterator>>> iterator = keysMap.iterator(); + while (resultSetKeyIterator.hasNext()) { + final StaticBuffer next = resultSetKeyIterator.next(); + try (final RecordIterator entries = resultSetKeyIterator.getEntries()) { + final Tuple2>> current = iterator.next(); + final ByteBuffer currentKey = current._1; + final Array> columnValues = current._2; + + final Iterator> columnIterator = columnValues.iterator(); + while (entries.hasNext()) { + final Entry entry = entries.next(); + final Tuple2 columnAndValue = columnIterator.next(); + + assertEquals(currentKey, next.asByteBuffer()); + assertEquals(columnAndValue._1, entry.getColumn().asByteBuffer()); + assertEquals(columnAndValue._2, entry.getValue().asByteBuffer()); + assertEquals(columnIterator.hasNext(), entries.hasNext()); + + // 10% of the time, don't complete the iteration + if (random.nextInt(10) == 0) { + break; + } + } + } + } + } + } + + @Test + public void testNoIterateColumns() throws IOException { + final Random random = new Random(); + + final Function1 randomLong = idx -> { + final ByteBuffer buffer = ByteBuffer.allocate(Long.BYTES).putLong(random.nextLong()); + buffer.flip(); + return buffer; + }; + + final Array>>> keysMap = Array.range(0, random.nextInt(100) + 100) + .map(randomLong) + .map(key -> Tuple.of(key, Array.rangeClosed(0, random.nextInt(100) + 1) + .map(idx -> Tuple.of(randomLong.apply(idx), randomLong.apply(idx))))); + + final Seq rows = keysMap.flatMap(tuple -> tuple._2.map(columnAndValue -> { + final Row row = mock(Row.class); + when(row.getBytes("key")).thenReturn(tuple._1); + when(row.getBytes("column1")).thenReturn(columnAndValue._1); + when(row.getBytes("value")).thenReturn(columnAndValue._2); + return row; + })); + + final ResultSet resultSet = mock(ResultSet.class); + when(resultSet.iterator()).thenReturn(rows.iterator()); + + final CQLColValGetter getter = new CQLColValGetter(new EntryMetaData[0]); + try (final CQLResultSetKeyIterator resultSetKeyIterator = new CQLResultSetKeyIterator(ALL_COLUMNS, getter, resultSet)) { + final Iterator>>> iterator = keysMap.iterator(); + while (resultSetKeyIterator.hasNext()) { + final StaticBuffer next = resultSetKeyIterator.next(); + assertEquals(iterator.next()._1, next.asByteBuffer()); + } + } + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLSSLStoreTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLSSLStoreTest.java new file mode 100644 index 0000000000..bc9ca03262 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLSSLStoreTest.java @@ -0,0 +1,47 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.testcategory.CassandraSSLTests; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +@Category({ CassandraSSLTests.class }) +public class CQLSSLStoreTest extends CQLStoreTest { + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(true); + } + + @Override + protected ModifiableConfiguration getBaseStorageConfiguration() { + return enableSSL(getCQLConfiguration(getClass().getSimpleName())); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager() throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLStoreTest.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLStoreTest.java new file mode 100644 index 0000000000..31916f9095 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CQLStoreTest.java @@ -0,0 +1,154 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CassandraStorageSetup.*; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.*; +import static org.junit.Assert.*; + +import java.util.Collections; +import java.util.Map; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.KeyColumnValueStoreTest; +import org.janusgraph.diskstorage.configuration.Configuration; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.janusgraph.diskstorage.keycolumnvalue.StoreFeatures; +import org.janusgraph.testcategory.OrderedKeyStoreTests; +import org.janusgraph.testcategory.UnorderedKeyStoreTests; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.collect.ImmutableMap; + +public class CQLStoreTest extends KeyColumnValueStoreTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(CQLStoreTest.class); + + private static final String TEST_CF_NAME = "testcf"; + private static final String DEFAULT_COMPRESSOR_PACKAGE = "org.apache.cassandra.io.compress"; + + @BeforeClass + public static void startCassandra() { + startCleanEmbedded(); + } + + protected ModifiableConfiguration getBaseStorageConfiguration() { + return getCQLConfiguration(getClass().getSimpleName()); + } + + private CQLStoreManager openStorageManager(final Configuration c) throws BackendException { + return new CQLStoreManager(c); + } + + @Override + public CQLStoreManager openStorageManager() throws BackendException { + return openStorageManager(getBaseStorageConfiguration()); + } + + @Test + @Category({ UnorderedKeyStoreTests.class }) + public void testUnorderedConfiguration() { + if (!this.manager.getFeatures().hasUnorderedScan()) { + LOGGER.warn( + "Can't test key-unordered features on incompatible store. " + + "This warning could indicate reduced test coverage and " + + "a broken JUnit configuration. Skipping test {}.", + this.name.getMethodName()); + return; + } + + final StoreFeatures features = this.manager.getFeatures(); + assertFalse(features.isKeyOrdered()); + assertFalse(features.hasLocalKeyPartition()); + } + + @Test + @Category({ OrderedKeyStoreTests.class }) + public void testOrderedConfiguration() { + if (!this.manager.getFeatures().hasOrderedScan()) { + LOGGER.warn( + "Can't test key-ordered features on incompatible store. " + + "This warning could indicate reduced test coverage and " + + "a broken JUnit configuration. Skipping test {}.", + this.name.getMethodName()); + return; + } + + final StoreFeatures features = this.manager.getFeatures(); + assertTrue(features.isKeyOrdered()); + } + + @Test + public void testDefaultCFCompressor() throws BackendException { + final String cf = TEST_CF_NAME + "_snappy"; + + final CQLStoreManager cqlStoreManager = openStorageManager(); + cqlStoreManager.openDatabase(cf); + + final Map defaultCfCompressionOps = new ImmutableMap.Builder() + .put("sstable_compression", DEFAULT_COMPRESSOR_PACKAGE + "." + CF_COMPRESSION_TYPE.getDefaultValue()) + .put("chunk_length_kb", "64") + .build(); + assertEquals(defaultCfCompressionOps, cqlStoreManager.getCompressionOptions(cf)); + } + + @Test + public void testCustomCFCompressor() throws BackendException { + final String cname = "DeflateCompressor"; + final int ckb = 128; + final String cf = TEST_CF_NAME + "_gzip"; + + final ModifiableConfiguration config = getBaseStorageConfiguration(); + config.set(CF_COMPRESSION_TYPE, cname); + config.set(CF_COMPRESSION_BLOCK_SIZE, ckb); + + final CQLStoreManager mgr = openStorageManager(config); + + // N.B.: clearStorage() truncates CFs but does not delete them + mgr.openDatabase(cf); + + final Map expected = ImmutableMap + . builder() + .put("sstable_compression", DEFAULT_COMPRESSOR_PACKAGE + "." + cname) + .put("chunk_length_kb", String.valueOf(ckb)) + .build(); + + assertEquals(expected, mgr.getCompressionOptions(cf)); + } + + @Test + public void testDisableCFCompressor() throws BackendException { + final String cf = TEST_CF_NAME + "_nocompress"; + + final ModifiableConfiguration config = getBaseStorageConfiguration(); + config.set(CF_COMPRESSION, false); + final CQLStoreManager mgr = openStorageManager(config); + + // N.B.: clearStorage() truncates CFs but does not delete them + mgr.openDatabase(cf); + + assertEquals(Collections.emptyMap(), mgr.getCompressionOptions(cf)); + } + + @Test + public void testTTLSupported() { + final StoreFeatures features = this.manager.getFeatures(); + assertTrue(features.hasCellTTL()); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CassandraStorageSetup.java b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CassandraStorageSetup.java new file mode 100644 index 0000000000..6d04a06c5f --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/diskstorage/cql/CassandraStorageSetup.java @@ -0,0 +1,153 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.diskstorage.cql; + +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.KEYSPACE; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.PROTOCOL_VERSION; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_ENABLED; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_TRUSTSTORE_LOCATION; +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.SSL_TRUSTSTORE_PASSWORD; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.CONNECTION_TIMEOUT; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.PAGE_SIZE; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.STORAGE_BACKEND; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.STORAGE_HOSTS; +import static org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration.buildGraphConfiguration; + +import java.io.File; +import java.io.IOException; +import java.time.Duration; + +import org.apache.commons.io.FileUtils; +import org.janusgraph.diskstorage.cassandra.utils.CassandraDaemonWrapper; +import org.janusgraph.diskstorage.configuration.ModifiableConfiguration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.common.base.Joiner; +import com.google.common.base.Preconditions; + +public class CassandraStorageSetup { + + private static final Logger LOGGER = LoggerFactory.getLogger(CassandraStorageSetup.class); + + private static final String TEST_STORAGE_HOST = "test.storage.host"; + + public static final String CONFDIR_SYSPROP = "test.cassandra.confdir"; + public static final String DATADIR_SYSPROP = "test.cassandra.datadir"; + + private static volatile Paths paths; + + /** + * Load cassandra.yaml and data paths from the environment or from default values if nothing is set in the environment, then delete all + * existing data, and finally start Cassandra. + *

+ * This method is idempotent. Calls after the first have no effect aside from logging statements. + */ + public static void startCleanEmbedded() { + startCleanEmbedded(false); + } + + public static void startCleanEmbedded(boolean force) { + if (force || System.getProperty(TEST_STORAGE_HOST) == null) { + final Paths p = getPaths(); + if (!CassandraDaemonWrapper.isStarted()) { + try { + FileUtils.deleteDirectory(new File(p.dataPath)); + } catch (final IOException e) { + throw new RuntimeException(e); + } + } + CassandraDaemonWrapper.start(p.yamlPath); + } + } + + private static synchronized Paths getPaths() { + if (null == paths) { + final String yamlPath = "file://" + loadAbsoluteDirectoryPath("conf", CONFDIR_SYSPROP, true) + File.separator + + "cassandra.yaml"; + final String dataPath = loadAbsoluteDirectoryPath("data", DATADIR_SYSPROP, false); + paths = new Paths(yamlPath, dataPath); + } + return paths; + } + + private static String loadAbsoluteDirectoryPath(final String name, final String prop, final boolean mustExistAndBeAbsolute) { + String s = System.getProperty(prop); + + if (null == s) { + s = Joiner.on(File.separator).join(System.getProperty("user.dir"), "target", "cassandra", "byteorderedpartitioner", name); + LOGGER.info("Set default Cassandra {} directory path {}", name, s); + } else { + LOGGER.info("Loaded Cassandra {} directory path {} from system property {}", new Object[] { name, s, prop }); + } + + if (mustExistAndBeAbsolute) { + final File dir = new File(s); + Preconditions.checkArgument(dir.isDirectory(), "Path %s must be a directory", s); + Preconditions.checkArgument(dir.isAbsolute(), "Path %s must be absolute", s); + } + + return s; + } + + public static ModifiableConfiguration getCQLConfiguration(final String keyspace) { + final ModifiableConfiguration config = buildGraphConfiguration(); + config.set(KEYSPACE, cleanKeyspaceName(keyspace)); + LOGGER.debug("Set keyspace name: {}", config.get(KEYSPACE)); + config.set(PAGE_SIZE, 500); + config.set(CONNECTION_TIMEOUT, Duration.ofSeconds(60L)); + config.set(STORAGE_BACKEND, "cql"); + // Set to 3 because we have a 2.1.9 database that only supports version 3, if we let it negotiate then there are spurious errors. + config.set(PROTOCOL_VERSION, 3); + if (System.getProperty(TEST_STORAGE_HOST) != null) { + config.set(STORAGE_HOSTS, new String[] { System.getProperty(TEST_STORAGE_HOST) }); + } + return config; + } + + public static ModifiableConfiguration enableSSL(final ModifiableConfiguration mc) { + mc.set(SSL_ENABLED, true); + mc.set(STORAGE_HOSTS, new String[] { "localhost" }); + mc.set(SSL_TRUSTSTORE_LOCATION, + Joiner.on(File.separator).join("target", "cassandra", "murmur-ssl", "conf", "test.truststore")); + mc.set(SSL_TRUSTSTORE_PASSWORD, "cassandra"); + return mc; + } + + /** + * Cassandra only accepts keyspace names 48 characters long or shorter made up of alphanumeric characters and underscores. + */ + private static String cleanKeyspaceName(final String raw) { + Preconditions.checkNotNull(raw); + Preconditions.checkArgument(0 < raw.length()); + + if (48 < raw.length() || raw.matches("^.*[^a-zA-Z0-9_].*$")) { + return "strhash" + String.valueOf(Math.abs(raw.hashCode())); + } else { + return raw; + } + } + + private static class Paths { + + private final String yamlPath; + private final String dataPath; + + public Paths(final String yamlPath, final String dataPath) { + this.yamlPath = yamlPath; + this.dataPath = dataPath; + } + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLEventualGraphTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLEventualGraphTest.java new file mode 100644 index 0000000000..27ca0b0310 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLEventualGraphTest.java @@ -0,0 +1,33 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphEventualGraphTest; +import org.junit.BeforeClass; + +public class CQLEventualGraphTest extends JanusGraphEventualGraphTest { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphCacheTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphCacheTest.java new file mode 100644 index 0000000000..71330fdecc --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphCacheTest.java @@ -0,0 +1,35 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.StorageSetup; +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphTest; +import org.junit.BeforeClass; + +public class CQLGraphCacheTest extends JanusGraphTest { + + @Override + public WriteConfiguration getConfiguration() { + return StorageSetup.addPermanentCache(CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName())); + } + + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphConcurrentTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphConcurrentTest.java new file mode 100644 index 0000000000..d887208425 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphConcurrentTest.java @@ -0,0 +1,37 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphConcurrentTest; +import org.janusgraph.testcategory.PerformanceTests; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; + +@Category({PerformanceTests.class}) +public class CQLGraphConcurrentTest extends JanusGraphConcurrentTest { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphIterativeTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphIterativeTest.java new file mode 100644 index 0000000000..f97cec795c --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphIterativeTest.java @@ -0,0 +1,44 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.configuration.BasicConfiguration; +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CQLStoreManager; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.diskstorage.keycolumnvalue.KeyColumnValueStoreManager; +import org.janusgraph.graphdb.JanusGraphIterativeBenchmark; +import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; +import org.junit.BeforeClass; + +public class CQLGraphIterativeTest extends JanusGraphIterativeBenchmark { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + @Override + public KeyColumnValueStoreManager openStorageManager() throws BackendException { + return new CQLStoreManager(new BasicConfiguration(GraphDatabaseConfiguration.ROOT_NS,getConfiguration(), BasicConfiguration.Restriction.NONE)); + } + + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphPerformanceMemoryTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphPerformanceMemoryTest.java new file mode 100644 index 0000000000..4709b55ae9 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphPerformanceMemoryTest.java @@ -0,0 +1,34 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphPerformanceMemoryTest; +import org.junit.BeforeClass; + +public class CQLGraphPerformanceMemoryTest extends JanusGraphPerformanceMemoryTest { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphSpeedTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphSpeedTest.java new file mode 100644 index 0000000000..4b706f1e66 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphSpeedTest.java @@ -0,0 +1,66 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.core.JanusGraphFactory; +import org.janusgraph.diskstorage.BackendException; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphSpeedTest; +import org.janusgraph.graphdb.SpeedTestSchema; +import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; +import org.janusgraph.graphdb.database.StandardJanusGraph; +import org.janusgraph.testcategory.PerformanceTests; +import org.junit.BeforeClass; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Category({PerformanceTests.class}) +public class CQLGraphSpeedTest extends JanusGraphSpeedTest { + + private static StandardJanusGraph graph; + private static SpeedTestSchema schema; + + private static final Logger log = LoggerFactory.getLogger(CQLGraphSpeedTest.class); + + public CQLGraphSpeedTest() throws BackendException { + super(CassandraStorageSetup.getCQLConfiguration(CQLGraphSpeedTest.class.getSimpleName()).getConfiguration()); + } + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } + + @Override + protected StandardJanusGraph getGraph() throws BackendException { + if (null == graph) { + GraphDatabaseConfiguration graphconfig = new GraphDatabaseConfiguration(conf); + graphconfig.getBackend().clearStorage(); + log.debug("Cleared backend storage"); + graph = (StandardJanusGraph)JanusGraphFactory.open(conf); + initializeGraph(graph); + } + return graph; + } + + @Override + protected SpeedTestSchema getSchema() { + if (null == schema) { + schema = SpeedTestSchema.get(); + } + return schema; + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphTest.java new file mode 100644 index 0000000000..ed5747c54c --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLGraphTest.java @@ -0,0 +1,58 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import static org.janusgraph.diskstorage.cql.CQLConfigOptions.KEYSPACE; +import static org.junit.Assert.*; + +import org.janusgraph.core.JanusGraphFactory; +import org.janusgraph.diskstorage.configuration.ConfigElement; +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.CassandraGraphTest; +import org.janusgraph.graphdb.configuration.GraphDatabaseConfiguration; +import org.janusgraph.graphdb.configuration.JanusGraphConstants; +import org.janusgraph.graphdb.database.StandardJanusGraph; +import org.junit.Test; + +public class CQLGraphTest extends CassandraGraphTest { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + @Test + public void testTitanGraphBackwardCompatibility() { + close(); + WriteConfiguration wc = getConfiguration(); + wc.set(ConfigElement.getPath(KEYSPACE), "titan"); + wc.set(ConfigElement.getPath(GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS), "x.x.x"); + + assertNull(wc.get(ConfigElement.getPath(GraphDatabaseConfiguration.INITIAL_JANUSGRAPH_VERSION), + GraphDatabaseConfiguration.INITIAL_JANUSGRAPH_VERSION.getDatatype())); + + assertFalse(JanusGraphConstants.TITAN_COMPATIBLE_VERSIONS.contains( + wc.get(ConfigElement.getPath(GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS), + GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS.getDatatype()))); + + wc.set(ConfigElement.getPath(GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS), "1.0.0"); + assertTrue(JanusGraphConstants.TITAN_COMPATIBLE_VERSIONS.contains( + wc.get(ConfigElement.getPath(GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS), + GraphDatabaseConfiguration.TITAN_COMPATIBLE_VERSIONS.getDatatype()))); + + graph = (StandardJanusGraph) JanusGraphFactory.open(wc); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOLAPTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOLAPTest.java new file mode 100644 index 0000000000..73e4868a1a --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOLAPTest.java @@ -0,0 +1,34 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.olap.OLAPTest; +import org.junit.BeforeClass; + +public class CQLOLAPTest extends OLAPTest { + + @Override + public WriteConfiguration getConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOperationCountingTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOperationCountingTest.java new file mode 100644 index 0000000000..a8a87f0fbc --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLOperationCountingTest.java @@ -0,0 +1,34 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphOperationCountingTest; +import org.junit.BeforeClass; + +public class CQLOperationCountingTest extends JanusGraphOperationCountingTest { + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } + + @Override + public WriteConfiguration getBaseConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + +} diff --git a/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLPartitionGraphTest.java b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLPartitionGraphTest.java new file mode 100644 index 0000000000..23a3a3e390 --- /dev/null +++ b/janusgraph-cql/src/test/java/org/janusgraph/graphdb/cql/CQLPartitionGraphTest.java @@ -0,0 +1,34 @@ +// Copyright 2017 JanusGraph Authors +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package org.janusgraph.graphdb.cql; + +import org.janusgraph.diskstorage.configuration.WriteConfiguration; +import org.janusgraph.diskstorage.cql.CassandraStorageSetup; +import org.janusgraph.graphdb.JanusGraphPartitionGraphTest; +import org.junit.BeforeClass; + +public class CQLPartitionGraphTest extends JanusGraphPartitionGraphTest { + + @BeforeClass + public static void beforeClass() { + CassandraStorageSetup.startCleanEmbedded(); + } + + @Override + public WriteConfiguration getBaseConfiguration() { + return CassandraStorageSetup.getCQLConfiguration(getClass().getSimpleName()).getConfiguration(); + } + +} diff --git a/janusgraph-cql/src/test/resources/cassandra/cassandra-env.sh b/janusgraph-cql/src/test/resources/cassandra/cassandra-env.sh new file mode 100644 index 0000000000..c296abfc8d --- /dev/null +++ b/janusgraph-cql/src/test/resources/cassandra/cassandra-env.sh @@ -0,0 +1,124 @@ +# 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. + +calculate_heap_size() +{ + case "`uname`" in + Linux) + system_memory_in_mb=`free -m | awk '/Mem:/ {print $2}'` + MAX_HEAP_SIZE=$((system_memory_in_mb / 2))M + return 0 + ;; + FreeBSD) + system_memory_in_bytes=`sysctl hw.physmem | awk '{print $2}'` + MAX_HEAP_SIZE=$((system_memory_in_bytes / 1024 / 1024 / 2))M + return 0 + ;; + *) + MAX_HEAP_SIZE=1024M + return 1 + ;; + esac +} + +# The amount of memory to allocate to the JVM at startup, you almost +# certainly want to adjust this for your environment. If left commented +# out, the heap size will be automatically determined by calculate_heap_size +# MAX_HEAP_SIZE="4G" + +if [ "x$MAX_HEAP_SIZE" = "x" ]; then + calculate_heap_size +fi + +# Specifies the default port over which Cassandra will be available for +# JMX connections. +JMX_PORT="9090" + +# To use mx4j, an HTML interface for JMX, add mx4j-tools.jar to the lib/ directory. +# By default mx4j listens on 0.0.0.0:8081. Uncomment the following lines to control +# its listen address and port. +#MX4J_ADDRESS="-Dmx4jaddress=0.0.0.0" +#MX4J_PORT="-Dmx4jport=8081" + + +# Here we create the arguments that will get passed to the jvm when +# starting cassandra. + +# enable assertions. disabling this in production will give a modest +# performance benefit (around 5%). +#JVM_OPTS="$JVM_OPTS -ea" + +# enable thread priorities, primarily so we can give periodic tasks +# a lower priority to avoid interfering with client workload +JVM_OPTS="$JVM_OPTS -XX:+UseThreadPriorities" +# allows lowering thread priority without being root. see +# http://tech.stolsvik.com/2010/01/linux-java-thread-priorities-workaround.html +JVM_OPTS="$JVM_OPTS -XX:ThreadPriorityPolicy=42" + +# min and max heap sizes should be set to the same value to avoid +# stop-the-world GC pauses during resize, and so that we can lock the +# heap in memory on startup to prevent any of it from being swapped +# out. +JVM_OPTS="$JVM_OPTS -Xms$MAX_HEAP_SIZE" +JVM_OPTS="$JVM_OPTS -Xmx$MAX_HEAP_SIZE" +JVM_OPTS="$JVM_OPTS -XX:+HeapDumpOnOutOfMemoryError" + +if [ "`uname`" = "Linux" ] ; then + # reduce the per-thread stack size to minimize the impact of Thrift + # thread-per-client. (Best practice is for client connections to + # be pooled anyway.) Only do so on Linux where it is known to be + # supported. + JVM_OPTS="$JVM_OPTS -Xss256k" +fi + +# GC tuning options +JVM_OPTS="$JVM_OPTS -XX:+UseParNewGC" +JVM_OPTS="$JVM_OPTS -XX:+UseConcMarkSweepGC" +JVM_OPTS="$JVM_OPTS -XX:+CMSParallelRemarkEnabled" +JVM_OPTS="$JVM_OPTS -XX:SurvivorRatio=8" +JVM_OPTS="$JVM_OPTS -XX:MaxTenuringThreshold=1" +JVM_OPTS="$JVM_OPTS -XX:CMSInitiatingOccupancyFraction=75" +JVM_OPTS="$JVM_OPTS -XX:+UseCMSInitiatingOccupancyOnly" + +# GC logging options -- uncomment to enable +# JVM_OPTS="$JVM_OPTS -XX:+PrintGCDetails" +# JVM_OPTS="$JVM_OPTS -XX:+PrintGCTimeStamps" +# JVM_OPTS="$JVM_OPTS -XX:+PrintClassHistogram" +# JVM_OPTS="$JVM_OPTS -XX:+PrintTenuringDistribution" +# JVM_OPTS="$JVM_OPTS -XX:+PrintGCApplicationStoppedTime" +# JVM_OPTS="$JVM_OPTS -Xloggc:/var/log/cassandra/gc.log" + +# Prefer binding to IPv4 network intefaces (when net.ipv6.bindv6only=1). See +# http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=6342561 (short version: +# comment out this entry to enable IPv6 support). +JVM_OPTS="$JVM_OPTS -Djava.net.preferIPv4Stack=true" + +# jmx: metrics and administration interface +# +# add this if you're having trouble connecting: +# JVM_OPTS="$JVM_OPTS -Djava.rmi.server.hostname=" +# +# see +# http://blogs.sun.com/jmxetc/entry/troubleshooting_connection_problems_in_jconsole +# for more on configuring JMX through firewalls, etc. (Short version: +# get it working with no firewall first.) + +# Disable JMX so multiple cassandras can run on 127.0.0.1, .2, .3, etc. +#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.port=$JMX_PORT" +#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.ssl=false" +#JVM_OPTS="$JVM_OPTS -Dcom.sun.management.jmxremote.authenticate=false" +#JVM_OPTS="$JVM_OPTS $MX4J_ADDRESS" +#JVM_OPTS="$JVM_OPTS $MX4J_PORT" diff --git a/janusgraph-cql/src/test/resources/cassandra/cassandra.in.sh b/janusgraph-cql/src/test/resources/cassandra/cassandra.in.sh new file mode 100644 index 0000000000..17e6e15918 --- /dev/null +++ b/janusgraph-cql/src/test/resources/cassandra/cassandra.in.sh @@ -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. + +if [ "x$CASSANDRA_HOME" = "x" ]; then + CASSANDRA_HOME=`dirname $0`/.. +fi + +# The directory where Cassandra's configs live (required) +if [ "x$CASSANDRA_CONF" = "x" ]; then + CASSANDRA_CONF=$CASSANDRA_HOME/conf +fi + +export MAX_HEAP_SIZE=256m + +# This can be the path to a jar file, or a directory containing the +# compiled classes. NOTE: This isn't needed by the startup script, +# it's just used here in constructing the classpath. +cassandra_bin=$CASSANDRA_HOME/build/classes +#cassandra_bin=$cassandra_home/build/cassandra.jar + +# JAVA_HOME can optionally be set here +#JAVA_HOME=/usr/local/jdk6 + +# The java classpath (required) +CLASSPATH=$CASSANDRA_CONF:$cassandra_bin + +for jar in $CASSANDRA_HOME/lib/*.jar; do + CLASSPATH=$CLASSPATH:$jar +done + +mkdir -p ${cassandra.dir} + +#export MAX_HEAP_SIZE=2048m diff --git a/janusgraph-cql/src/test/resources/cassandra/cassandra.yaml b/janusgraph-cql/src/test/resources/cassandra/cassandra.yaml new file mode 100644 index 0000000000..449582dd92 --- /dev/null +++ b/janusgraph-cql/src/test/resources/cassandra/cassandra.yaml @@ -0,0 +1,664 @@ +# Cassandra storage config YAML + +# NOTE: +# See http://wiki.apache.org/cassandra/StorageConfiguration for +# full explanations of configuration directives +# /NOTE + +# The name of the cluster. This is mainly used to prevent machines in +# one logical cluster from joining another. +cluster_name: 'Cassandra testing instance' + +# This defines the number of tokens randomly assigned to this node on the ring +# The more tokens, relative to other nodes, the larger the proportion of data +# that this node will store. You probably want all nodes to have the same number +# of tokens assuming they have equal hardware capability. +# +# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, +# and will use the initial_token as described below. +# +# Specifying initial_token will override this setting. +# +# If you already have a cluster with 1 token per node, and wish to migrate to +# multiple tokens per node, see http://wiki.apache.org/cassandra/Operations +#num_tokens: 256 + +# initial_token allows you to specify tokens manually. While you can use # it with +# vnodes (num_tokens > 1, above) -- in which case you should provide a +# comma-separated list -- it's primarily used when adding nodes # to legacy clusters +# that do not have vnodes enabled. +# initial_token: + +initial_token: ${cassandra.token} + +${cassandra.num.tokens} + +# May either be "true" or "false" to enable globally, or contain a list +# of data centers to enable per-datacenter. +# hinted_handoff_enabled: DC1,DC2 +# See http://wiki.apache.org/cassandra/HintedHandoff +hinted_handoff_enabled: true +# this defines the maximum amount of time a dead host will have hints +# generated. After it has been dead this long, new hints for it will not be +# created until it has been seen alive and gone down again. +max_hint_window_in_ms: 10800000 # 3 hours +# Maximum throttle in KBs per second, per delivery thread. This will be +# reduced proportionally to the number of nodes in the cluster. (If there +# are two nodes in the cluster, each delivery thread will use the maximum +# rate; if there are three, each will throttle to half of the maximum, +# since we expect two nodes to be delivering hints simultaneously.) +hinted_handoff_throttle_in_kb: 1024 +# Number of threads with which to deliver hints; +# Consider increasing this number when you have multi-dc deployments, since +# cross-dc handoff tends to be slower +max_hints_delivery_threads: 2 + +# Maximum throttle in KBs per second, total. This will be +# reduced proportionally to the number of nodes in the cluster. +batchlog_replay_throttle_in_kb: 1024 + +# Authentication backend, implementing IAuthenticator; used to identify users +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator, +# PasswordAuthenticator}. +# +# - AllowAllAuthenticator performs no checks - set it to disable authentication. +# - PasswordAuthenticator relies on username/password pairs to authenticate +# users. It keeps usernames and hashed passwords in system_auth.credentials table. +# Please increase system_auth keyspace replication factor if you use this authenticator. +authenticator: AllowAllAuthenticator + +# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer, +# CassandraAuthorizer}. +# +# - AllowAllAuthorizer allows any action to any user - set it to disable authorization. +# - CassandraAuthorizer stores permissions in system_auth.permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +authorizer: AllowAllAuthorizer + +# Validity period for permissions cache (fetching permissions can be an +# expensive operation depending on the authorizer, CassandraAuthorizer is +# one example). Defaults to 2000, set to 0 to disable. +# Will be disabled automatically for AllowAllAuthorizer. +permissions_validity_in_ms: 2000 + +# The partitioner is responsible for distributing groups of rows (by +# partition key) across nodes in the cluster. You should leave this +# alone for new clusters. The partitioner can NOT be changed without +# reloading all data, so when upgrading you should set this to the +# same partitioner you were already using. +# +# Besides Murmur3Partitioner, partitioners included for backwards +# compatibility include RandomPartitioner, ByteOrderedPartitioner, and +# OrderPreservingPartitioner. +# +partitioner: ${cassandra.partitioner} + +# Directories where Cassandra should store data on disk. Cassandra +# will spread data evenly across them, subject to the granularity of +# the configured compaction strategy. +data_file_directories: + - ${cassandra.dir}/data + +# commit log +commitlog_directory: ${cassandra.dir}/commitlog + +# policy for data disk failures: +# stop_paranoid: shut down gossip and Thrift even for single-sstable errors. +# stop: shut down gossip and Thrift, leaving the node effectively dead, but +# can still be inspected via JMX. +# best_effort: stop using the failed disk and respond to requests based on +# remaining available sstables. This means you WILL see obsolete +# data at CL.ONE! +# ignore: ignore fatal errors and let requests fail, as in pre-1.2 Cassandra +disk_failure_policy: stop + +# policy for commit disk failures: +# stop: shut down gossip and Thrift, leaving the node effectively dead, but +# can still be inspected via JMX. +# stop_commit: shutdown the commit log, letting writes collect but +# continuing to service reads, as in pre-2.0.5 Cassandra +# ignore: ignore fatal errors and let the batches fail +commit_failure_policy: stop + +# Maximum size of the key cache in memory. +# +# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the +# minimum, sometimes more. The key cache is fairly tiny for the amount of +# time it saves, so it's worthwhile to use it at large numbers. +# The row cache saves even more time, but must contain the entire row, +# so it is extremely space-intensive. It's best to only use the +# row cache if you have hot rows or static rows. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache. +key_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# save the key cache. Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 14400 or 4 hours. +key_cache_save_period: 14400 + +# Number of keys from the key cache to save +# Disabled by default, meaning all keys are going to be saved +# key_cache_keys_to_save: 100 + +# Maximum size of the row cache in memory. +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is 0, to disable row caching. +row_cache_size_in_mb: 0 + +# Duration in seconds after which Cassandra should +# safe the row cache. Caches are saved to saved_caches_directory as specified +# in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 0 to disable saving the row cache. +row_cache_save_period: 0 + +# Number of keys from the row cache to save +# Disabled by default, meaning all keys are going to be saved +# row_cache_keys_to_save: 100 + +# The off-heap memory allocator. Affects storage engine metadata as +# well as caches. Experiments show that JEMAlloc saves some memory +# than the native GCC allocator (i.e., JEMalloc is more +# fragmentation-resistant). +# +# Supported values are: NativeAllocator, JEMallocAllocator +# +# If you intend to use JEMallocAllocator you have to install JEMalloc as library and +# modify cassandra-env.sh as directed in the file. +# +# Defaults to NativeAllocator +# memory_allocator: NativeAllocator + +# saved caches +saved_caches_directory: ${cassandra.dir}/saved_caches + +# commitlog_sync may be either "periodic" or "batch." +# When in batch mode, Cassandra won't ack writes until the commit log +# has been fsynced to disk. It will wait up to +# commitlog_sync_batch_window_in_ms milliseconds for other writes, before +# performing the sync. +# +# commitlog_sync: batch +# commitlog_sync_batch_window_in_ms: 50 +# +# the other option is "periodic" where writes may be acked immediately +# and the CommitLog is simply synced every commitlog_sync_period_in_ms +# milliseconds. By default this allows 1024*(CPU cores) pending +# entries on the commitlog queue. If you are writing very large blobs, +# you should reduce that; 16*cores works reasonably well for 1MB blobs. +# It should be at least as large as the concurrent_writes setting. +commitlog_sync: periodic +commitlog_sync_period_in_ms: 10000 +# commitlog_periodic_queue_size: + +# The size of the individual commitlog file segments. A commitlog +# segment may be archived, deleted, or recycled once all the data +# in it (potentially from each columnfamily in the system) has been +# flushed to sstables. +# +# The default size is 32, which is almost always fine, but if you are +# archiving commitlog segments (see commitlog_archiving.properties), +# then you probably want a finer granularity of archiving; 8 or 16 MB +# is reasonable. +commitlog_segment_size_in_mb: 32 + +# any class that implements the SeedProvider interface and has a +# constructor that takes a Map of parameters will do. +seed_provider: + # Addresses of hosts that are deemed contact points. + # Cassandra nodes use this list of hosts to find each other and learn + # the topology of the ring. You must change this if you are running + # multiple nodes! + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + # seeds is actually a comma-delimited list of addresses. + # Ex: ",," + - seeds: ${cassandra.seed} + +# For workloads with more data than can fit in memory, Cassandra's +# bottleneck will be reads that need to fetch data from +# disk. "concurrent_reads" should be set to (16 * number_of_drives) in +# order to allow the operations to enqueue low enough in the stack +# that the OS and drives can reorder them. +# +# On the other hand, since writes are almost never IO bound, the ideal +# number of "concurrent_writes" is dependent on the number of cores in +# your system; (8 * number_of_cores) is a good rule of thumb. +concurrent_reads: 32 +concurrent_writes: 32 + +# Total memory to use for sstable-reading buffers. Defaults to +# the smaller of 1/4 of heap or 512MB. +# file_cache_size_in_mb: 512 + +# Total memory to use for memtables. Cassandra will flush the largest +# memtable when this much memory is used. +# If omitted, Cassandra will set it to 1/4 of the heap. +# memtable_total_space_in_mb: 2048 + +# Total space to use for commitlogs. Since commitlog segments are +# mmapped, and hence use up address space, the default size is 32 +# on 32-bit JVMs, and 1024 on 64-bit JVMs. +# +# If space gets above this value (it will round up to the next nearest +# segment multiple), Cassandra will flush every dirty CF in the oldest +# segment and remove it. So a small total commitlog space will tend +# to cause more flush activity on less-active columnfamilies. +# commitlog_total_space_in_mb: 4096 + +# This sets the amount of memtable flush writer threads. These will +# be blocked by disk io, and each one will hold a memtable in memory +# while blocked. If you have a large heap and many data directories, +# you can increase this value for better flush performance. +# By default this will be set to the amount of data directories defined. +#memtable_flush_writers: 1 + +# Whether to, when doing sequential writing, fsync() at intervals in +# order to force the operating system to flush the dirty +# buffers. Enable this to avoid sudden dirty buffer flushing from +# impacting read latencies. Almost always a good idea on SSDs; not +# necessarily on platters. +trickle_fsync: false +trickle_fsync_interval_in_kb: 10240 + +# TCP port, for commands and data +storage_port: 7000 + +# SSL port, for encrypted communication. Unused unless enabled in +# encryption_options +ssl_storage_port: 7001 + +# Address to bind to and tell other Cassandra nodes to connect to. You +# _must_ change this if you want multiple nodes to be able to +# communicate! +# +# Leaving it blank leaves it up to InetAddress.getLocalHost(). This +# will always do the Right Thing _if_ the node is properly configured +# (hostname, name resolution, etc), and the Right Thing is to use the +# address associated with the hostname (it might not be). +# +# Setting this to 0.0.0.0 is always wrong. +listen_address: ${cassandra.ip} + +# Address to broadcast to other Cassandra nodes +# Leaving this blank will set it to the same value as listen_address +# broadcast_address: 1.2.3.4 + +# Internode authentication backend, implementing IInternodeAuthenticator; +# used to allow/disallow connections from peer nodes. +# internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator + +# Whether to start the native transport server. +# Please note that the address on which the native transport is bound is the +# same as the rpc_address. The port however is different and specified below. +start_native_transport: true +# port for the CQL native transport to listen for clients on +native_transport_port: 9042 +# The maximum threads for handling requests when the native transport is used. +# This is similar to rpc_max_threads though the default differs slightly (and +# there is no native_transport_min_threads, idle threads will always be stopped +# after 30 seconds). +# native_transport_max_threads: 128 +# +# The maximum size of allowed frame. Frame (requests) larger than this will +# be rejected as invalid. The default is 256MB. +# native_transport_max_frame_size_in_mb: 256 + +# Whether to start the thrift rpc server. +start_rpc: true + +# The address to bind the Thrift RPC service and native transport +# server -- clients connect here. +# +# Leaving this blank has the same effect it does for ListenAddress, +# (i.e. it will be based on the configured hostname of the node). +# +# Note that unlike ListenAddress above, it is allowed to specify 0.0.0.0 +# here if you want to listen on all interfaces, but that will break clients +# that rely on node auto-discovery. +rpc_address: ${cassandra.ip} +# port for Thrift to listen for clients on +rpc_port: 9160 + +# enable or disable keepalive on rpc/native connections +rpc_keepalive: true + +# Cassandra provides two out-of-the-box options for the RPC Server: +# +# sync -> One thread per thrift connection. For a very large number of clients, memory +# will be your limiting factor. On a 64 bit JVM, 180KB is the minimum stack size +# per thread, and that will correspond to your use of virtual memory (but physical memory +# may be limited depending on use of stack space). +# +# hsha -> Stands for "half synchronous, half asynchronous." All thrift clients are handled +# asynchronously using a small number of threads that does not vary with the amount +# of thrift clients (and thus scales well to many clients). The rpc requests are still +# synchronous (one thread per active request). +# +# The default is sync because on Windows hsha is about 30% slower. On Linux, +# sync/hsha performance is about the same, with hsha of course using less memory. +# +# Alternatively, can provide your own RPC server by providing the fully-qualified class name +# of an o.a.c.t.TServerFactory that can create an instance of it. +rpc_server_type: sync + +# Uncomment rpc_min|max_thread to set request pool size limits. +# +# Regardless of your choice of RPC server (see above), the number of maximum requests in the +# RPC thread pool dictates how many concurrent requests are possible (but if you are using the sync +# RPC server, it also dictates the number of clients that can be connected at all). +# +# The default is unlimited and thus provides no protection against clients overwhelming the server. You are +# encouraged to set a maximum that makes sense for you in production, but do keep in mind that +# rpc_max_threads represents the maximum number of client requests this server may execute concurrently. +# +# rpc_min_threads: 16 +# rpc_max_threads: 2048 + +# uncomment to set socket buffer sizes on rpc connections +# rpc_send_buff_size_in_bytes: +# rpc_recv_buff_size_in_bytes: + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# See: +# /proc/sys/net/core/wmem_max +# /proc/sys/net/core/rmem_max +# /proc/sys/net/ipv4/tcp_wmem +# /proc/sys/net/ipv4/tcp_wmem +# and: man tcp +# internode_send_buff_size_in_bytes: +# internode_recv_buff_size_in_bytes: + +# Frame size for thrift (maximum message length). +thrift_framed_transport_size_in_mb: 15 + +# Set to true to have Cassandra create a hard link to each sstable +# flushed or streamed locally in a backups/ subdirectory of the +# keyspace data. Removing these links is the operator's +# responsibility. +incremental_backups: false + +# Whether or not to take a snapshot before each compaction. Be +# careful using this option, since Cassandra won't clean up the +# snapshots for you. Mostly useful if you're paranoid when there +# is a data format change. +snapshot_before_compaction: false + +# Whether or not a snapshot is taken of the data before keyspace truncation +# or dropping of column families. The STRONGLY advised default of true +# should be used to provide data safety. If you set this flag to false, you will +# lose data on truncation or drop. +auto_snapshot: true + +# When executing a scan, within or across a partition, we need to keep the +# tombstones seen in memory so we can return them to the coordinator, which +# will use them to make sure other replicas also know about the deleted rows. +# With workloads that generate a lot of tombstones, this can cause performance +# problems and even exaust the server heap. +# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) +# Adjust the thresholds here if you understand the dangers and want to +# scan more tombstones anyway. These thresholds may also be adjusted at runtime +# using the StorageService mbean. +tombstone_warn_threshold: 1000 +tombstone_failure_threshold: 100000 + +# Add column indexes to a row after its contents reach this size. +# Increase if your column values are large, or if you have a very large +# number of columns. The competing causes are, Cassandra has to +# deserialize this much of the row to read a single column, so you want +# it to be small - at least if you do many partial-row reads - but all +# the index data is read for each access, so you don't want to generate +# that wastefully either. +column_index_size_in_kb: 64 + + +# Log WARN on any batch size exceeding this value. 5kb per batch by default. +# Caution should be taken on increasing the size of this threshold as it can lead to node instability. +batch_size_warn_threshold_in_kb: 5 + +# Number of simultaneous compactions to allow, NOT including +# validation "compactions" for anti-entropy repair. Simultaneous +# compactions can help preserve read performance in a mixed read/write +# workload, by mitigating the tendency of small sstables to accumulate +# during a single long running compactions. The default is usually +# fine and if you experience problems with compaction running too +# slowly or too fast, you should look at +# compaction_throughput_mb_per_sec first. +# +# concurrent_compactors defaults to the number of cores. +# Uncomment to make compaction mono-threaded, the pre-0.8 default. +#concurrent_compactors: 1 + +# Throttles compaction to the given total throughput across the entire +# system. The faster you insert data, the faster you need to compact in +# order to keep the sstable count down, but in general, setting this to +# 16 to 32 times the rate you are inserting data is more than sufficient. +# Setting this to 0 disables throttling. Note that this account for all types +# of compaction, including validation compaction. +compaction_throughput_mb_per_sec: 16 + +# Throttles all outbound streaming file transfers on this node to the +# given total throughput in Mbps. This is necessary because Cassandra does +# mostly sequential IO when streaming data during bootstrap or repair, which +# can lead to saturating the network connection and degrading rpc performance. +# When unset, the default is 200 Mbps or 25 MB/s. +# stream_throughput_outbound_megabits_per_sec: 200 + +# How long the coordinator should wait for read operations to complete +read_request_timeout_in_ms: 30000 +# How long the coordinator should wait for seq or index scans to complete +range_request_timeout_in_ms: 30000 +# How long the coordinator should wait for writes to complete +write_request_timeout_in_ms: 30000 +# How long a coordinator should continue to retry a CAS operation +# that contends with other proposals for the same row +cas_contention_timeout_in_ms: 30000 +# How long the coordinator should wait for truncates to complete +# (This can be much longer, because unless auto_snapshot is disabled +# we need to flush first so we can snapshot before removing the data.) +truncate_request_timeout_in_ms: 60000 +# The default timeout for other, miscellaneous operations +request_timeout_in_ms: 30000 + +# Enable operation timeout information exchange between nodes to accurately +# measure request timeouts. If disabled, replicas will assume that requests +# were forwarded to them instantly by the coordinator, which means that +# under overload conditions we will waste that much extra time processing +# already-timed-out requests. +# +# Warning: before enabling this property make sure to ntp is installed +# and the times are synchronized between the nodes. +cross_node_timeout: false + +# Enable socket timeout for streaming operation. +# When a timeout occurs during streaming, streaming is retried from the start +# of the current file. This _can_ involve re-streaming an important amount of +# data, so you should avoid setting the value too low. +# Default value is 0, which never timeout streams. +# streaming_socket_timeout_in_ms: 0 + +# phi value that must be reached for a host to be marked down. +# most users should never need to adjust this. +# phi_convict_threshold: 8 + +# endpoint_snitch -- Set this to a class that implements +# IEndpointSnitch. The snitch has two functions: +# - it teaches Cassandra enough about your network topology to route +# requests efficiently +# - it allows Cassandra to spread replicas around your cluster to avoid +# correlated failures. It does this by grouping machines into +# "datacenters" and "racks." Cassandra will do its best not to have +# more than one replica on the same "rack" (which may not actually +# be a physical location) +# +# IF YOU CHANGE THE SNITCH AFTER DATA IS INSERTED INTO THE CLUSTER, +# YOU MUST RUN A FULL REPAIR, SINCE THE SNITCH AFFECTS WHERE REPLICAS +# ARE PLACED. +# +# Out of the box, Cassandra provides +# - SimpleSnitch: +# Treats Strategy order as proximity. This can improve cache +# locality when disabling read repair. Only appropriate for +# single-datacenter deployments. +# - GossipingPropertyFileSnitch +# This should be your go-to snitch for production use. The rack +# and datacenter for the local node are defined in +# cassandra-rackdc.properties and propagated to other nodes via +# gossip. If cassandra-topology.properties exists, it is used as a +# fallback, allowing migration from the PropertyFileSnitch. +# - PropertyFileSnitch: +# Proximity is determined by rack and data center, which are +# explicitly configured in cassandra-topology.properties. +# - Ec2Snitch: +# Appropriate for EC2 deployments in a single Region. Loads Region +# and Availability Zone information from the EC2 API. The Region is +# treated as the datacenter, and the Availability Zone as the rack. +# Only private IPs are used, so this will not work across multiple +# Regions. +# - Ec2MultiRegionSnitch: +# Uses public IPs as broadcast_address to allow cross-region +# connectivity. (Thus, you should set seed addresses to the public +# IP as well.) You will need to open the storage_port or +# ssl_storage_port on the public IP firewall. (For intra-Region +# traffic, Cassandra will switch to the private IP after +# establishing a connection.) +# - RackInferringSnitch: +# Proximity is determined by rack and data center, which are +# assumed to correspond to the 3rd and 2nd octet of each node's IP +# address, respectively. Unless this happens to match your +# deployment conventions, this is best used as an example of +# writing a custom Snitch class and is provided in that spirit. +# +# You can use a custom Snitch by setting this to the full class name +# of the snitch, which will be assumed to be on your classpath. +endpoint_snitch: SimpleSnitch + +# controls how often to perform the more expensive part of host score +# calculation +dynamic_snitch_update_interval_in_ms: 100 +# controls how often to reset all host scores, allowing a bad host to +# possibly recover +dynamic_snitch_reset_interval_in_ms: 600000 +# if set greater than zero and read_repair_chance is < 1.0, this will allow +# 'pinning' of replicas to hosts in order to increase cache capacity. +# The badness threshold will control how much worse the pinned host has to be +# before the dynamic snitch will prefer other replicas over it. This is +# expressed as a double which represents a percentage. Thus, a value of +# 0.2 means Cassandra would continue to prefer the static snitch values +# until the pinned host was 20% worse than the fastest. +dynamic_snitch_badness_threshold: 0.1 + +# request_scheduler -- Set this to a class that implements +# RequestScheduler, which will schedule incoming client requests +# according to the specific policy. This is useful for multi-tenancy +# with a single Cassandra cluster. +# NOTE: This is specifically for requests from the client and does +# not affect inter node communication. +# org.apache.cassandra.scheduler.NoScheduler - No scheduling takes place +# org.apache.cassandra.scheduler.RoundRobinScheduler - Round robin of +# client requests to a node with a separate queue for each +# request_scheduler_id. The scheduler is further customized by +# request_scheduler_options as described below. +request_scheduler: org.apache.cassandra.scheduler.NoScheduler + +# Scheduler Options vary based on the type of scheduler +# NoScheduler - Has no options +# RoundRobin +# - throttle_limit -- The throttle_limit is the number of in-flight +# requests per client. Requests beyond +# that limit are queued up until +# running requests can complete. +# The value of 80 here is twice the number of +# concurrent_reads + concurrent_writes. +# - default_weight -- default_weight is optional and allows for +# overriding the default which is 1. +# - weights -- Weights are optional and will default to 1 or the +# overridden default_weight. The weight translates into how +# many requests are handled during each turn of the +# RoundRobin, based on the scheduler id. +# +# request_scheduler_options: +# throttle_limit: 80 +# default_weight: 5 +# weights: +# Keyspace1: 1 +# Keyspace2: 5 + +# request_scheduler_id -- An identifier based on which to perform +# the request scheduling. Currently the only valid option is keyspace. +# request_scheduler_id: keyspace + +# Enable or disable inter-node encryption +# Default settings are TLS v1, RSA 1024-bit keys (it is imperative that +# users generate their own keys) TLS_RSA_WITH_AES_128_CBC_SHA as the cipher +# suite for authentication, key exchange and encryption of the actual data transfers. +# Use the DHE/ECDHE ciphers if running in FIPS 140 compliant mode. +# NOTE: No custom encryption options are enabled at the moment +# The available internode options are : all, none, dc, rack +# +# If set to dc cassandra will encrypt the traffic between the DCs +# If set to rack cassandra will encrypt the traffic between the racks +# +# The passwords used in these options must match the passwords used when generating +# the keystore and truststore. For instructions on generating these files, see: +# http://download.oracle.com/javase/6/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# +server_encryption_options: + internode_encryption: none + keystore: conf/.keystore + keystore_password: cassandra + truststore: conf/.truststore + truststore_password: cassandra + # More advanced defaults below: + # protocol: TLS + # algorithm: SunX509 + # store_type: JKS + # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA,TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA] + # require_client_auth: false + +# enable or disable client/server encryption. + +client_encryption_options: + enabled: ${cassandra.clientenc.enabled} + keystore: ${cassandra.clientenc.kspath} + keystore_password: ${cassandra.clientenc.kspw} + require_client_auth: ${cassandra.clientenc.ccert} + # Set trustore and truststore_password if require_client_auth is true + truststore: ${cassandra.clientenc.tspath} #conf/.truststore + truststore_password: ${cassandra.clientenc.tspw} + # More advanced defaults below: + # protocol: TLS + # algorithm: SunX509 + # store_type: JKS + cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA] + # cipher_suites: [TLS_RSA_WITH_AES_128_CBC_SHA,TLS_RSA_WITH_AES_256_CBC_SHA,TLS_DHE_RSA_WITH_AES_128_CBC_SHA,TLS_DHE_RSA_WITH_AES_256_CBC_SHA,TLS_ECDHE_ + +# internode_compression controls whether traffic between nodes is +# compressed. +# can be: all - all traffic is compressed +# dc - traffic between different datacenters is compressed +# none - nothing is compressed. +internode_compression: all + +# Enable or disable tcp_nodelay for inter-dc communication. +# Disabling it will result in larger (but fewer) network packets being sent, +# reducing overhead from the TCP protocol itself, at the cost of increasing +# latency if you block for cross-datacenter responses. +inter_dc_tcp_nodelay: false diff --git a/janusgraph-cql/src/test/resources/cassandra/log4j-empty.properties b/janusgraph-cql/src/test/resources/cassandra/log4j-empty.properties new file mode 100644 index 0000000000..e69de29bb2 diff --git a/janusgraph-cql/src/test/resources/cassandra/log4j-server.properties b/janusgraph-cql/src/test/resources/cassandra/log4j-server.properties new file mode 100644 index 0000000000..5940a91b40 --- /dev/null +++ b/janusgraph-cql/src/test/resources/cassandra/log4j-server.properties @@ -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. + +# for production, you should probably set pattern to %c instead of %l. +# (%l is slower.) + +# output messages into a rolling log file as well as stdout +log4j.rootLogger=INFO,stdout,R + +# stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n + +# rolling log file +log4j.appender.R=org.apache.log4j.RollingFileAppender +log4j.appender.R.maxFileSize=20MB +log4j.appender.R.maxBackupIndex=50 +log4j.appender.R.layout=org.apache.log4j.PatternLayout +log4j.appender.R.layout.ConversionPattern=%5p [%t] %d{ISO8601} %F (line %L) %m%n +# Edit the next line to point to your logs directory +log4j.appender.R.File=${cassandra.dir}/system.log + +# Application logging options +#log4j.logger.org.apache.cassandra=DEBUG +#log4j.logger.org.apache.cassandra.graph=DEBUG +#log4j.logger.org.apache.cassandra.service.StorageProxy=DEBUG + diff --git a/janusgraph-cql/src/test/resources/cassandra/log4j-tools.properties b/janusgraph-cql/src/test/resources/cassandra/log4j-tools.properties new file mode 100644 index 0000000000..a8f4d9f05b --- /dev/null +++ b/janusgraph-cql/src/test/resources/cassandra/log4j-tools.properties @@ -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. + +# for production, you should probably set the root to INFO +# and the pattern to %c instead of %l. (%l is slower.) + +# output messages into a rolling log file as well as stdout +log4j.rootLogger=WARN,stderr + +# stderr +log4j.appender.stderr=org.apache.log4j.ConsoleAppender +log4j.appender.stderr.target=System.err +log4j.appender.stderr.layout=org.apache.log4j.PatternLayout +log4j.appender.stderr.layout.ConversionPattern=%5p %d{HH:mm:ss,SSS} %m%n diff --git a/janusgraph-cql/src/test/resources/keystores/README.md b/janusgraph-cql/src/test/resources/keystores/README.md new file mode 100644 index 0000000000..961fb897be --- /dev/null +++ b/janusgraph-cql/src/test/resources/keystores/README.md @@ -0,0 +1,3 @@ +This keypair is just for testing. Under no circumstances should it be +used on any system that matters. It's been public on github and +compromised since the day it was generated. \ No newline at end of file diff --git a/janusgraph-cql/src/test/resources/keystores/test.keystore b/janusgraph-cql/src/test/resources/keystores/test.keystore new file mode 100644 index 0000000000..729078c9e2 Binary files /dev/null and b/janusgraph-cql/src/test/resources/keystores/test.keystore differ diff --git a/janusgraph-cql/src/test/resources/keystores/test.truststore b/janusgraph-cql/src/test/resources/keystores/test.truststore new file mode 100644 index 0000000000..005300c797 Binary files /dev/null and b/janusgraph-cql/src/test/resources/keystores/test.truststore differ diff --git a/janusgraph-cql/src/test/resources/log4j.properties b/janusgraph-cql/src/test/resources/log4j.properties new file mode 100644 index 0000000000..3f01de576a --- /dev/null +++ b/janusgraph-cql/src/test/resources/log4j.properties @@ -0,0 +1,15 @@ +log4j.appender.FILE=org.apache.log4j.FileAppender +log4j.appender.FILE.File=${project.build.directory}/test.log +log4j.appender.FILE.Threshold=TRACE +log4j.appender.FILE.layout=org.apache.log4j.PatternLayout +log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%20.20t] %-5p %30.30c: %m%n + +log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender +log4j.appender.CONSOLE.Threshold=INFO +log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout +log4j.appender.CONSOLE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss.SSS} [%20.20t] %-5p %30.30c: %m%n + +log4j.rootLogger=INFO,FILE,CONSOLE + +log4j.logger.org.apache.cassandra=WARN +log4j.logger.com.datastax.driver=WARN \ No newline at end of file diff --git a/janusgraph-cql/src/test/resources/properties/byteorderedpartitioner.properties b/janusgraph-cql/src/test/resources/properties/byteorderedpartitioner.properties new file mode 100644 index 0000000000..f3253c1517 --- /dev/null +++ b/janusgraph-cql/src/test/resources/properties/byteorderedpartitioner.properties @@ -0,0 +1,10 @@ +cassandra.ip = 127.0.0.1 +cassandra.seed = 127.0.0.1 +# This must be set to prevent Cassandra from self-killing with BOP: +# https://issues.apache.org/jira/browse/CASSANDRA-5414 +cassandra.token = 0000000000000000000000000000000000 +cassandra.num.tokens = +cassandra.partitioner = org.apache.cassandra.dht.ByteOrderedPartitioner +cassandra.dir = ${project.build.directory}/cassandra/${test.byteorderedpartitioner} +cassandra.clientenc.enabled = false +cassandra.clientenc.ccert = false \ No newline at end of file diff --git a/janusgraph-cql/src/test/resources/properties/murmur-ssl.properties b/janusgraph-cql/src/test/resources/properties/murmur-ssl.properties new file mode 100644 index 0000000000..5b67ec1e20 --- /dev/null +++ b/janusgraph-cql/src/test/resources/properties/murmur-ssl.properties @@ -0,0 +1,10 @@ +cassandra.ip = 127.0.0.1 +cassandra.seed = 127.0.0.1 +cassandra.token = +cassandra.num.tokens = num_tokens: 4 +cassandra.partitioner = org.apache.cassandra.dht.Murmur3Partitioner +cassandra.dir = ${project.build.directory}/cassandra/${test.murmur-ssl} +cassandra.clientenc.enabled = true +cassandra.clientenc.ccert = false +cassandra.clientenc.kspath = ${project.build.directory}/cassandra/${test.murmur-ssl}/conf/test.keystore +cassandra.clientenc.kspw = cassandra diff --git a/janusgraph-cql/src/test/resources/properties/murmur.properties b/janusgraph-cql/src/test/resources/properties/murmur.properties new file mode 100644 index 0000000000..7723e62b71 --- /dev/null +++ b/janusgraph-cql/src/test/resources/properties/murmur.properties @@ -0,0 +1,8 @@ +cassandra.ip = 127.0.0.1 +cassandra.seed = 127.0.0.1 +cassandra.token = +cassandra.num.tokens = num_tokens: 4 +cassandra.partitioner = org.apache.cassandra.dht.Murmur3Partitioner +cassandra.dir = ${project.build.directory}/cassandra/${test.murmur} +cassandra.clientenc.enabled = false +cassandra.clientenc.ccert = false diff --git a/janusgraph-dist/src/assembly/cfilter/conf/janusgraph-cql.properties b/janusgraph-dist/src/assembly/cfilter/conf/janusgraph-cql.properties new file mode 100644 index 0000000000..31af17e180 --- /dev/null +++ b/janusgraph-dist/src/assembly/cfilter/conf/janusgraph-cql.properties @@ -0,0 +1,19 @@ +# JanusGraph configuration sample: Cassandra over a socket +# +# This file connects to a Cassandra daemon running on localhost via +# Thrift. Cassandra must already be started before starting JanusGraph +# with this file. + +gremlin.graph=org.janusgraph.core.JanusGraphFactory + +#JANUSGRAPHCFG{storage.backend=cql} + +#JANUSGRAPHCFG{storage.hostname=127.0.0.1} + +#JANUSGRAPHCFG{cache.db-cache = true} + +#JANUSGRAPHCFG{cache.db-cache-clean-wait = 20} + +#JANUSGRAPHCFG{cache.db-cache-time = 180000} + +#JANUSGRAPHCFG{cache.db-cache-size = 0.5} diff --git a/janusgraph-test/src/main/java/org/janusgraph/diskstorage/DistributedStoreManagerTest.java b/janusgraph-test/src/main/java/org/janusgraph/diskstorage/DistributedStoreManagerTest.java index dabb89626f..ed4fcc5b19 100644 --- a/janusgraph-test/src/main/java/org/janusgraph/diskstorage/DistributedStoreManagerTest.java +++ b/janusgraph-test/src/main/java/org/janusgraph/diskstorage/DistributedStoreManagerTest.java @@ -16,32 +16,33 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; +import static org.junit.Assume.assumeTrue; import java.util.List; -import org.junit.Test; -import org.junit.experimental.categories.Category; - import org.janusgraph.diskstorage.common.DistributedStoreManager; import org.janusgraph.diskstorage.common.DistributedStoreManager.Deployment; import org.janusgraph.diskstorage.keycolumnvalue.KeyColumnValueStore; import org.janusgraph.diskstorage.keycolumnvalue.KeyRange; import org.janusgraph.testcategory.OrderedKeyStoreTests; +import org.junit.Test; +import org.junit.experimental.categories.Category; public abstract class DistributedStoreManagerTest { - + protected T manager; protected KeyColumnValueStore store; - + @Test @Category({ OrderedKeyStoreTests.class }) public void testGetDeployment() { assertEquals(Deployment.LOCAL, manager.getDeployment()); } - + @Test @Category({ OrderedKeyStoreTests.class }) public void testGetLocalKeyPartition() throws BackendException { + assumeTrue(manager.getFeatures().hasLocalKeyPartition()); List local = manager.getLocalKeyPartition(); assertNotNull(local); assertEquals(1, local.size()); diff --git a/janusgraph-test/src/main/java/org/janusgraph/graphdb/JanusGraphBaseTest.java b/janusgraph-test/src/main/java/org/janusgraph/graphdb/JanusGraphBaseTest.java index 0cec67a1f1..a927ff4c33 100644 --- a/janusgraph-test/src/main/java/org/janusgraph/graphdb/JanusGraphBaseTest.java +++ b/janusgraph-test/src/main/java/org/janusgraph/graphdb/JanusGraphBaseTest.java @@ -92,10 +92,10 @@ public void setUp() throws Exception { this.config = getConfiguration(); TestGraphConfigs.applyOverrides(config); Preconditions.checkNotNull(config); + logManagers = new HashMap(); clearGraph(config); readConfig = new BasicConfiguration(GraphDatabaseConfiguration.ROOT_NS, config, BasicConfiguration.Restriction.NONE); open(config); - logManagers = new HashMap(); } public void open(WriteConfiguration config) { diff --git a/pom.xml b/pom.xml index d59614b9f0..e8a06bda48 100644 --- a/pom.xml +++ b/pom.xml @@ -127,6 +127,7 @@ janusgraph-core janusgraph-test janusgraph-berkeleyje + janusgraph-cql janusgraph-cassandra janusgraph-hadoop-parent janusgraph-hbase-parent