Skip to content

Commit

Permalink
Added dtype test case for spanner
Browse files Browse the repository at this point in the history
  • Loading branch information
taher-koitawala committed Jun 15, 2023
1 parent c8e5f5f commit 6555aa8
Show file tree
Hide file tree
Showing 3 changed files with 141 additions and 18 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import io.trino.testing.QueryRunner;
import io.trino.tpch.TpchTable;
import org.intellij.lang.annotations.Language;
import org.jetbrains.annotations.NotNull;

import java.util.HashMap;
import java.util.Map;
Expand All @@ -44,10 +45,10 @@ public static DistributedQueryRunner createSpannerQueryRunner(
TestingSpannerInstance instance,
Map<String, String> extraProperties,
Map<String, String> connectorProperties,
Iterable<TpchTable<?>> tables)
Iterable<TpchTable<?>> tables, boolean addTpcDsTables)
throws Exception
{
return createSpannerQueryRunner(instance, extraProperties, ImmutableMap.of(), connectorProperties, tables, runner -> {});
return createSpannerQueryRunner(instance, extraProperties, ImmutableMap.of(), connectorProperties, tables, runner -> {}, addTpcDsTables);
}

public static DistributedQueryRunner createSpannerQueryRunner(
Expand All @@ -56,7 +57,8 @@ public static DistributedQueryRunner createSpannerQueryRunner(
Map<String, String> coordinatorProperties,
Map<String, String> connectorProperties,
Iterable<TpchTable<?>> tables,
Consumer<QueryRunner> moreSetup)
Consumer<QueryRunner> moreSetup,
boolean addTpcDsTables)
throws Exception
{
DistributedQueryRunner queryRunner = null;
Expand Down Expand Up @@ -88,13 +90,12 @@ public static DistributedQueryRunner createSpannerQueryRunner(
connectorProperties.putIfAbsent("spanner.emulated.host", "localhost:9010");*/
queryRunner.installPlugin(new SpannerPlugin());
queryRunner.createCatalog("spanner", "spanner", connectorProperties);
MaterializedResult execute1 = queryRunner.execute("create table emp WITH (PRIMARY_KEYS = ARRAY['id']) as select 1 as id,'T' as name");
System.out.println(execute1);
MaterializedResult execute2 = queryRunner.execute("select * from emp");
System.out.println(execute2);
copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables);
MaterializedResult execute = queryRunner.execute("SHOW TABLES FROM spanner.default");
System.out.println(execute);
if (addTpcDsTables) {
copyTpchTables(queryRunner, "tpch", TINY_SCHEMA_NAME, createSession(), tables);
MaterializedResult execute = queryRunner.execute("SHOW TABLES FROM spanner.default");
System.out.println(execute);
}

/*
MaterializedResult rows = queryRunner.execute("SELECT * FROM spanner.default.customer");
System.out.println(rows);
Expand All @@ -121,11 +122,7 @@ public static Session createSession()
public static void main(String[] args)
throws Exception
{
DistributedQueryRunner queryRunner = createSpannerQueryRunner(
new TestingSpannerInstance(),
ImmutableMap.of("http-server.http.port", "8080"),
ImmutableMap.of(),
TpchTable.getTables());
DistributedQueryRunner queryRunner = getSpannerQueryRunner();

queryRunner.installPlugin(new JmxPlugin());
queryRunner.createCatalog("jmx", "jmx");
Expand All @@ -135,6 +132,17 @@ public static void main(String[] args)
log.info("\n====\n%s\n====", queryRunner.getCoordinator().getBaseUrl());
}

@NotNull
public static DistributedQueryRunner getSpannerQueryRunner()
throws Exception
{
return createSpannerQueryRunner(
new TestingSpannerInstance(),
ImmutableMap.of("http-server.http.port", "8080"),
ImmutableMap.of(),
TpchTable.getTables(), false);
}

private static void copyTpchTables(
QueryRunner queryRunner,
String sourceCatalog,
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,104 @@
package io.trino.plugin.spanner;

import com.google.common.collect.ImmutableMap;
import io.trino.Session;
import io.trino.spi.type.BigintType;
import io.trino.testing.AbstractTestQueryFramework;
import io.trino.testing.QueryRunner;
import io.trino.testing.datatype.CreateAndInsertDataSetup;
import io.trino.testing.datatype.CreateAsSelectDataSetup;
import io.trino.testing.datatype.DataSetup;
import io.trino.testing.datatype.SqlDataTypeTest;
import io.trino.testing.sql.JdbcSqlExecutor;
import io.trino.testing.sql.TemporaryRelation;
import io.trino.testing.sql.TrinoSqlExecutor;
import io.trino.tpch.TpchTable;
import org.testng.annotations.Test;

import java.util.Properties;
import java.util.stream.Collectors;
import java.util.stream.IntStream;

import static io.trino.spi.type.BooleanType.BOOLEAN;

public class TestSpannerDataTypesMapping
extends AbstractTestQueryFramework
{
protected TestingSpannerInstance spannerInstance;

@Override
protected QueryRunner createQueryRunner()
throws Exception
{
spannerInstance = closeAfterClass(new TestingSpannerInstance());
return SpannerQueryRunner.createSpannerQueryRunner(
spannerInstance,
ImmutableMap.of("http-server.http.port", "8080"),
ImmutableMap.of(),
TpchTable.getTables(), false);
}

@Test
public void testBoolean()
{
SqlDataTypeTest.create()
.addRoundTrip("bool", "true", BOOLEAN)
.addRoundTrip("bool", "false", BOOLEAN)
.addRoundTrip("bool", "NULL", BOOLEAN, "CAST(NULL AS BOOLEAN)")
.addRoundTrip("int64", "1", BigintType.BIGINT, "CAST(1 as BIGINT)")
.execute(getQueryRunner(), spannerCreateAndInsert("test_boolean"))
.execute(getQueryRunner(), trinoCreateAsSelect("test_boolean"))
.execute(getQueryRunner(), trinoCreateAndInsert("test_boolean"));
}

private DataSetup spannerCreateAndInsert(String tableNamePrefix)
{
JdbcSqlExecutor jdbcSqlExecutor = new JdbcSqlExecutor(spannerInstance.getJdbcUrl(), new Properties());
return inputs -> {

String primaryKey = String.format("col_%s", inputs.size() - 1);

jdbcSqlExecutor.execute("CREATE TABLE %s (%s) PRIMARY KEY (%s)"
.formatted(
tableNamePrefix,
IntStream.range(0, inputs.size())
.mapToObj(f -> String.format("col_%s %s", f, inputs.get(f).getDeclaredType().get()))
.collect(Collectors.joining(", ")),
primaryKey));
return new TemporaryRelation()
{
@Override
public String getName()
{
return tableNamePrefix;
}

@Override
public void close()
{

}
};
};
}

private DataSetup trinoCreateAsSelect(String tableNamePrefix)
{
return trinoCreateAsSelect(getSession(), tableNamePrefix);
}

private DataSetup trinoCreateAsSelect(Session session, String tableNamePrefix)
{
return new CreateAsSelectDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix);
}

private DataSetup trinoCreateAndInsert(String tableNamePrefix)
{
return trinoCreateAndInsert(getSession(), tableNamePrefix);
}

private DataSetup trinoCreateAndInsert(Session session, String tableNamePrefix)
{
return new CreateAndInsertDataSetup(new TrinoSqlExecutor(getQueryRunner(), session), tableNamePrefix);
}
}
Original file line number Diff line number Diff line change
@@ -1,22 +1,33 @@
package io.trino.plugin.spanner;

import com.google.common.collect.ImmutableMap;
import io.trino.plugin.spanner.SpannerPlugin;
import io.trino.spi.Plugin;
import io.trino.spi.connector.ConnectorFactory;
import io.trino.testing.TestingConnectorContext;
import org.testcontainers.utility.DockerImageName;
import org.testng.annotations.Test;

import java.util.concurrent.ExecutionException;

import static com.google.common.collect.Iterables.getOnlyElement;

public class TestSpannerPlugin
{
@Test
public void testCreateConnector()
throws Exception
{
Plugin plugin = new SpannerPlugin();
ConnectorFactory factory = getOnlyElement(plugin.getConnectorFactories());
factory.create("test", ImmutableMap.of("connection-url", "jdbc:cloudspanner://0.0.0.0:9010/projects/spanner-project/instances/spanner-instance/databases/spanner-database;autoConfigEmulator=true"), new TestingConnectorContext()).shutdown();
TestingSpannerInstance instance = new TestingSpannerInstance();
factory.create("test", ImmutableMap.of(
"spanner.credentials.file", "credentials.json",
"spanner.instanceId", instance.getInstanceId()
, "spanner.projectId", instance.getProjectId()
, "spanner.database", instance.getDatabaseId()
, "spanner.emulated", "true"
, "spanner.emulated.host", instance.getHost()
),
new TestingConnectorContext()).shutdown();
instance.close();
}
}

0 comments on commit 6555aa8

Please sign in to comment.