Skip to content

Commit

Permalink
Add DataStax Enterprise as a Cassandra test environment
Browse files Browse the repository at this point in the history
  • Loading branch information
kamil-rafalko authored and ebyhr committed Aug 16, 2022
1 parent ab32ff0 commit fd7826e
Show file tree
Hide file tree
Showing 3 changed files with 78 additions and 5 deletions.
Expand Up @@ -20,6 +20,7 @@
import com.datastax.oss.driver.api.core.config.ProgrammaticDriverConfigLoaderBuilder;
import com.datastax.oss.driver.api.core.cql.ResultSet;
import com.datastax.oss.driver.api.core.cql.Row;
import com.google.common.collect.ImmutableMap;
import com.google.common.io.Resources;
import io.airlift.json.JsonCodec;
import io.airlift.log.Logger;
Expand All @@ -32,6 +33,7 @@
import java.net.InetSocketAddress;
import java.nio.file.Path;
import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeoutException;

import static com.datastax.oss.driver.api.core.config.DefaultDriverOption.CONTROL_CONNECTION_AGREEMENT_TIMEOUT;
Expand Down Expand Up @@ -72,16 +74,24 @@ public CassandraServer()

public CassandraServer(String imageName)
throws Exception
{
this(imageName, ImmutableMap.of(), "/etc/cassandra/cassandra.yaml", "cu-cassandra.yaml");
}

public CassandraServer(String imageName, Map<String, String> environmentVariables, String configPath, String configFileName)
throws Exception
{
log.info("Starting cassandra...");

this.dockerContainer = new GenericContainer<>(imageName)
.withExposedPorts(PORT)
.withCopyFileToContainer(forHostPath(prepareCassandraYaml()), "/etc/cassandra/cassandra.yaml");
.withCopyFileToContainer(forHostPath(prepareCassandraYaml(configFileName)), configPath)
.withEnv(environmentVariables)
.withStartupTimeout(java.time.Duration.ofMinutes(10));
this.dockerContainer.start();

ProgrammaticDriverConfigLoaderBuilder driverConfigLoaderBuilder = DriverConfigLoader.programmaticBuilder();
driverConfigLoaderBuilder.withDuration(REQUEST_TIMEOUT, java.time.Duration.ofSeconds(12));
driverConfigLoaderBuilder.withDuration(REQUEST_TIMEOUT, java.time.Duration.ofSeconds(30));
driverConfigLoaderBuilder.withString(PROTOCOL_VERSION, ProtocolVersion.V3.name());
driverConfigLoaderBuilder.withDuration(CONTROL_CONNECTION_AGREEMENT_TIMEOUT, java.time.Duration.ofSeconds(30));
// allow the retrieval of metadata for the system keyspaces
Expand Down Expand Up @@ -111,18 +121,18 @@ public CassandraServer(String imageName)
this.session = session;
}

private static String prepareCassandraYaml()
private static String prepareCassandraYaml(String fileName)
throws IOException
{
String original = Resources.toString(getResource("cu-cassandra.yaml"), UTF_8);
String original = Resources.toString(getResource(fileName), UTF_8);

Path tmpDirPath = createTempDirectory(null);
Path dataDir = tmpDirPath.resolve("data");
createDirectory(dataDir);

String modified = original.replaceAll("\\$\\{data_directory\\}", dataDir.toAbsolutePath().toString());

File yamlFile = tmpDirPath.resolve("cu-cassandra.yaml").toFile();
File yamlFile = tmpDirPath.resolve(fileName).toFile();
yamlFile.deleteOnExit();
writeString(yamlFile.toPath(), modified, UTF_8);

Expand Down
@@ -0,0 +1,43 @@
/*
* 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 io.trino.plugin.cassandra;

import com.google.common.collect.ImmutableMap;
import io.trino.testing.QueryRunner;

import java.sql.Timestamp;
import java.util.Map;

import static io.trino.plugin.cassandra.CassandraQueryRunner.createCassandraQueryRunner;
import static io.trino.plugin.cassandra.CassandraTestingUtils.createTestTables;

public class TestDatastaxConnectorSmokeTest
extends BaseCassandraConnectorSmokeTest
{
@Override
protected QueryRunner createQueryRunner()
throws Exception
{
CassandraServer server = closeAfterClass(new CassandraServer(
"datastax/dse-server:6.8.25",
Map.of(
"DS_LICENSE", "accept",
"DC", "datacenter1"),
"/config/cassandra.yaml",
"cassandra-dse.yaml"));
CassandraSession session = server.getSession();
createTestTables(session, KEYSPACE, Timestamp.from(TIMESTAMP_VALUE.toInstant()));
return createCassandraQueryRunner(server, ImmutableMap.of(), ImmutableMap.of(), REQUIRED_TPCH_TABLES);
}
}
20 changes: 20 additions & 0 deletions plugin/trino-cassandra/src/test/resources/cassandra-dse.yaml
@@ -0,0 +1,20 @@
endpoint_snitch: SimpleSnitch

commitlog_sync: periodic
commitlog_sync_period_in_ms: 10000

partitioner: org.apache.cassandra.dht.Murmur3Partitioner

seed_provider:
- class_name: org.apache.cassandra.locator.SimpleSeedProvider
parameters:
- seeds: "127.0.0.1"

native_transport_port: 9142

read_request_timeout_in_ms: 30000
range_request_timeout_in_ms: 30000
write_request_timeout_in_ms: 30000
cas_contention_timeout_in_ms: 30000
truncate_request_timeout_in_ms: 60000
request_timeout_in_ms: 30000

0 comments on commit fd7826e

Please sign in to comment.